From 1fe9f17e2873ddc78ba65301f32674e566446ef9 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 9 Dec 2024 17:47:41 -0800 Subject: [PATCH 01/49] add itemized file document and partition document --- core/workflow-core/build.sbt | 4 +- .../storage/result/ItemizedFileDocument.scala | 202 ++++++++++++++++++ .../storage/result/PartitionDocument.scala | 78 +++++++ .../result/ItemizedFileDocumentSpec.scala | 121 +++++++++++ 4 files changed, 404 insertions(+), 1 deletion(-) create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index 299aab37bd3..78dedbc56da 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -123,5 +123,7 @@ libraryDependencies ++= Seq( "com.typesafe.scala-logging" %% "scala-logging" % "3.9.5", // Scala Logging "org.eclipse.jgit" % "org.eclipse.jgit" % "5.13.0.202109080827-r", // jgit "org.yaml" % "snakeyaml" % "1.30", // yaml reader (downgrade to 1.30 due to dropwizard 1.3.23 required by amber) - "org.apache.commons" % "commons-vfs2" % "2.9.0" // for FileResolver throw VFS-related exceptions + "org.apache.commons" % "commons-vfs2" % "2.9.0", // for FileResolver throw VFS-related exceptions + "com.twitter" % "chill-java" % "0.10.0", // for Kryo serde/deserde + "com.twitter" %% "chill" % "0.10.0", // for Kyro serde/deserde ) \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala new file mode 100644 index 00000000000..0560c275137 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala @@ -0,0 +1,202 @@ +package edu.uci.ics.amber.core.storage.result + +import com.twitter.chill.{KryoPool, ScalaKryoInstantiator} +import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} +import org.apache.commons.vfs2.{FileObject, VFS} + +import java.io.{DataOutputStream, InputStream} +import java.net.URI +import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.collection.mutable.ArrayBuffer + +object ItemizedFileDocument { + // Initialize KryoPool as a static object + private val kryoPool = KryoPool.withByteArrayOutputStream(10, new ScalaKryoInstantiator) +} + +/** + * ItemizedFileDocument provides methods to read/write items to a file located on the filesystem. + * All methods are THREAD-SAFE, implemented using a read-write lock: + * - 1 writer at a time: only 1 thread can acquire the write lock. + * - n readers at a time: multiple threads can acquire the read lock. + * + * The type parameter T specifies the iterable data item stored in the file. + * + * @param uri the identifier of the file. If the file doesn't physically exist, ItemizedFileDocument will create it during construction. + */ +class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) + extends VirtualDocument[T] + with BufferedItemWriter[T] { + + val file: FileObject = VFS.getManager.resolveFile(uri.toString) + val lock = new ReentrantReadWriteLock() + + // Buffer to store items before flushing + private val buffer = new ArrayBuffer[T]() + override val bufferSize: Int = 1024 + + // Register a shutdown hook to delete the file when the JVM exits + sys.addShutdownHook { + withWriteLock { + if (file.exists()) { + file.delete() + } + } + } + + // Utility function to wrap code block with read lock + private def withReadLock[M](block: => M): M = { + lock.readLock().lock() + try { + block + } finally { + lock.readLock().unlock() + } + } + + // Utility function to wrap code block with write lock + private def withWriteLock[M](block: => M): M = { + lock.writeLock().lock() + try { + block + } finally { + lock.writeLock().unlock() + } + } + + // Check and create the file if it does not exist + withWriteLock { + if (!file.exists()) { + val parentDir = file.getParent + if (parentDir != null && !parentDir.exists()) { + parentDir.createFolder() // Create all necessary parent directories + } + file.createFile() // Create the file if it does not exist + } + } + + /** + * Append the content in the given object to the ItemizedFileDocument. This method is THREAD-SAFE. + * Each record will be stored as . + * + * @param item the content to append + */ + override def append(item: T): Unit = + withWriteLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() + } + } + + /** + * Write buffered items to the file and clear the buffer. + */ + private def flushBuffer(): Unit = + withWriteLock { + val outStream = file.getContent.getOutputStream(true) + val dataOutStream = new DataOutputStream(outStream) + try { + buffer.foreach { item => + val serializedBytes = ItemizedFileDocument.kryoPool.toBytesWithClass(item) + dataOutStream.writeInt(serializedBytes.length) + dataOutStream.write(serializedBytes) + } + buffer.clear() + } finally { + dataOutStream.close() + outStream.close() + } + } + + /** + * Open the writer. Initializes the buffer. + */ + override def open(): Unit = + withWriteLock { + buffer.clear() + } + + /** + * Close the writer, flushing any remaining buffered items to the file. + */ + override def close(): Unit = + withWriteLock { + if (buffer.nonEmpty) { + flushBuffer() + } + } + + /** + * Put one item into the buffer. Flushes if the buffer is full. + * + * @param item the data item to be written + */ + override def putOne(item: T): Unit = append(item) + + /** + * Remove one item from the buffer. This does not affect items already written to the file. + * + * @param item the item to remove + */ + override def removeOne(item: T): Unit = + withWriteLock { + buffer -= item + } + + /** + * Get the ith data item. The returned value will be deserialized using Kryo. + * + * @param i index starting from 0 + * @return data item of type T + */ + override def getItem(i: Int): T = { + val iterator = get() + iterator.drop(i).next() + } + + /** + * Get an iterator of data items of type T. Each returned item will be deserialized using Kryo. + * + * @return an iterator that returns data items of type T + */ + override def get(): Iterator[T] = { + lazy val input = new com.twitter.chill.Input(file.getContent.getInputStream) + new Iterator[T] { + var record: T = internalNext() + + private def internalNext(): T = { + try { + val len = input.readInt() + val bytes = input.readBytes(len) + ItemizedFileDocument.kryoPool.fromBytes(bytes).asInstanceOf[T] + } catch { + case _: Throwable => + input.close() + null + } + } + + override def next(): T = { + val currentRecord = record + record = internalNext() + currentRecord + } + + override def hasNext: Boolean = record != null + } + } + + /** + * Physically remove the file specified by the URI. This method is THREAD-SAFE. + */ + override def clear(): Unit = + withWriteLock { + if (!file.exists()) { + throw new RuntimeException(s"File $uri doesn't exist") + } + file.delete() + } + + override def getURI: URI = uri +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala new file mode 100644 index 00000000000..46b5d6c1273 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala @@ -0,0 +1,78 @@ +package edu.uci.ics.amber.core.storage.result + +import edu.uci.ics.amber.core.storage.model.VirtualDocument + +import java.net.URI + +/** + * PartitionDocument is a storage object that consists #numOfPartition physical files as its underlying data storage. + * Each underlying file's URI is in the format of {partitionDocumentURI}_{index}. + * + * PartitionDocument only support getting the FileDocument that corresponds to the single partition either by index or by iterator. + * To write over the partition, you should get the FileDocument first, then call write-related methods over it. FileDocument guarantees the thread-safe read/write. + * + * The Type parameter T is used to specify the type of data item stored in the partition + * @param uri the id of this partition document. Note that this URI does not physically corresponds to a file. + * @param numOfPartition number of partitions + */ +class PartitionDocument[T >: Null <: AnyRef](val uri: URI, val numOfPartition: Int) + extends VirtualDocument[ItemizedFileDocument[T]] { + + /** + * Utility functions to generate the partition URI by index + * @param i index of the partition + * @return the URI of the partition + */ + private def getPartitionURI(i: Int): URI = { + if (i < 0 || i >= numOfPartition) { + throw new RuntimeException(f"Index $i out of bound") + } + new URI(s"${uri}_$i") + } + + override def getURI: URI = + throw new RuntimeException( + "Partition Document doesn't physically exist. It is invalid to acquire its URI" + ) + + /** + * Get the partition by index i. + * This method is THREAD-UNSAFE, as multiple threads can get any partition by index. But the returned FileDocument is thread-safe + * @param i index starting from 0 + * @return FileDocument corresponds to the certain partition + */ + override def getItem(i: Int): ItemizedFileDocument[T] = { + new ItemizedFileDocument(getPartitionURI(i)) + } + + /** + * Get the iterator of partitions. + * This method is THREAD-UNSAFE, as multiple threads can get the iterator and loop through all partitions. But the returned FileDocument is thread-safe + * @return an iterator that return the FileDocument corresponds to the certain partition + */ + override def get(): Iterator[ItemizedFileDocument[T]] = + new Iterator[ItemizedFileDocument[T]] { + private var i: Int = 0 + + override def hasNext: Boolean = i < numOfPartition + + override def next(): ItemizedFileDocument[T] = { + if (!hasNext) { + throw new NoSuchElementException("No more partitions") + } + val document = new ItemizedFileDocument[T](getPartitionURI(i)) + i += 1 + document + } + } + + /** + * Remove all partitions. + * This method is THREAD-UNSAFE. But FileDocument's remove is thread-safe + */ + override def clear(): Unit = { + for (i <- 0 until numOfPartition) { + getItem(i).clear() + } + } +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala new file mode 100644 index 00000000000..6e1c0f1ec72 --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala @@ -0,0 +1,121 @@ +package edu.uci.ics.amber.storage.result + +import edu.uci.ics.amber.core.storage.result.ItemizedFileDocument + +import java.net.URI +import java.nio.file.{Files, Paths} +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.ScalaFutures.convertScalaFuture +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future +import scala.util.Using + +class ItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { + + var tempFileURI: URI = _ + var fileDocument: ItemizedFileDocument[String] = _ + + val initialContent = "Initial Content" + val newContent = "New Content" + + before { + // Generate a path for a temporary file + val tempPath = Files.createTempFile("", "") + tempFileURI = tempPath.toUri + fileDocument = new ItemizedFileDocument(tempFileURI) + + // Write initial content to file + fileDocument.putOne(initialContent) + fileDocument.close() + } + + after { + // Delete the temporary file + Files.deleteIfExists(Paths.get(tempFileURI)) + } + + "ItemizedFileDocument" should "allow writing and flushing buffered items" in { + // Add items to the buffer + fileDocument.open() + fileDocument.putOne("Buffered Item 1") + fileDocument.putOne("Buffered Item 2") + + // Force a flush + fileDocument.close() + + // Verify the items using the itemized get method + fileDocument.getItem(0) should equal(initialContent) + fileDocument.getItem(1) should equal("Buffered Item 1") + fileDocument.getItem(2) should equal("Buffered Item 2") + } + + it should "correctly flush the buffer when it reaches the buffer size" in { + val largeBuffer = (1 to fileDocument.bufferSize).map(i => s"Item $i") + + fileDocument.open() + largeBuffer.foreach(item => fileDocument.putOne(item)) + fileDocument.close() + + val items = fileDocument.get().toList + + items should contain(initialContent) + largeBuffer.foreach { item => + items should contain(item) + } + } + + it should "allow removing items from the buffer" in { + fileDocument.open() + fileDocument.putOne("Item to keep") + fileDocument.putOne("Item to remove") + fileDocument.removeOne("Item to remove") + fileDocument.close() + + val items = fileDocument.get().toList + + items should contain("Item to keep") + items should not contain "Item to remove" + } + + it should "handle concurrent buffered writes safely" in { + val numberOfThreads = 5 + val futures = (1 to numberOfThreads).map { i => + Future { + fileDocument.putOne(s"Content from thread $i") + } + } + + Future + .sequence(futures) + .map { _ => + fileDocument.close() + val items = fileDocument.get().toList + + items should contain(initialContent) + (1 to numberOfThreads).foreach { i => + items should contain(s"Content from thread $i") + } + } + .futureValue + } + + it should "handle writing after reopening the file" in { + fileDocument.open() + fileDocument.putOne("First Write") + fileDocument.close() + + // Reopen and write again + fileDocument.open() + fileDocument.putOne("Second Write") + fileDocument.close() + + val items = fileDocument.get().toList + + items should contain(initialContent) + items should contain("First Write") + items should contain("Second Write") + } +} From 219b82dd834ed827aff9fb0b89bcd62c21e30918 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 9 Dec 2024 23:05:15 -0800 Subject: [PATCH 02/49] add unit test for PartitionDocument --- .../storage/result/PartitionDocument.scala | 6 +- .../result/PartitionDocumentSpec.scala | 118 ++++++++++++++++++ 2 files changed, 121 insertions(+), 3 deletions(-) create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala index 46b5d6c1273..5974dc645a3 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala @@ -12,10 +12,10 @@ import java.net.URI * To write over the partition, you should get the FileDocument first, then call write-related methods over it. FileDocument guarantees the thread-safe read/write. * * The Type parameter T is used to specify the type of data item stored in the partition - * @param uri the id of this partition document. Note that this URI does not physically corresponds to a file. + * @param id the id of this partition document. * @param numOfPartition number of partitions */ -class PartitionDocument[T >: Null <: AnyRef](val uri: URI, val numOfPartition: Int) +class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) extends VirtualDocument[ItemizedFileDocument[T]] { /** @@ -27,7 +27,7 @@ class PartitionDocument[T >: Null <: AnyRef](val uri: URI, val numOfPartition: I if (i < 0 || i >= numOfPartition) { throw new RuntimeException(f"Index $i out of bound") } - new URI(s"${uri}_$i") + new URI(s"${id}_partition$i") } override def getURI: URI = diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala new file mode 100644 index 00000000000..d1bdbebc0cd --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala @@ -0,0 +1,118 @@ +package edu.uci.ics.amber.core.storage.result + +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.ScalaFutures.convertScalaFuture +import org.scalatest.matchers.should.Matchers + +import java.net.URI +import java.nio.file.{Files, Paths} +import scala.util.Using + +class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { + + var partitionDocument: PartitionDocument[String] = _ + val numOfPartitions = 3 + val partitionId = + Files.createTempDirectory("partition_doc_test").toUri.toString + "/test_partition" + + before { + // Initialize the PartitionDocument with a base ID and number of partitions + partitionDocument = new PartitionDocument[String](partitionId, numOfPartitions) + } + + after { + // Clean up all partitions after each test + partitionDocument.clear() + for (i <- 0 until numOfPartitions) { + Files.deleteIfExists(Paths.get(new URI(s"${partitionId}_partition$i"))) + } + Files.deleteIfExists(Paths.get(new URI(partitionId).getPath.stripSuffix("/test_partition"))) + } + + "PartitionDocument" should "create and write to each partition" in { + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + fileDoc.open() + fileDoc.putOne(s"Data for partition $i") + fileDoc.close() + } + + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + val items = fileDoc.get().toList + items should contain(s"Data for partition $i") + } + } + + it should "read from multiple partitions" in { + // Write some data to each partition + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + fileDoc.open() + fileDoc.putOne(s"Content in partition $i") + fileDoc.close() + } + + // Read and verify data from each partition + val partitionIterator = partitionDocument.get() + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionIterator.next() + val items = fileDoc.get().toList + items should contain(s"Content in partition $i") + } + } + + it should "clear all partitions" in { + // Write some data to each partition + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + fileDoc.open() + fileDoc.putOne(s"Some data in partition $i") + fileDoc.close() + } + + // Clear all partitions + partitionDocument.clear() + + // Verify that each partition is empty + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + val items = fileDoc.get().toList + items should be(empty) + } + } + + it should "handle concurrent writes to different partitions" in { + import scala.concurrent.ExecutionContext.Implicits.global + import scala.concurrent.Future + + val futures = (0 until numOfPartitions).map { i => + Future { + val fileDoc = partitionDocument.getItem(i) + fileDoc.open() + fileDoc.putOne(s"Concurrent write to partition $i") + fileDoc.close() + } + } + + Future.sequence(futures).futureValue + + // Verify data written concurrently + for (i <- 0 until numOfPartitions) { + val fileDoc = partitionDocument.getItem(i) + val items = fileDoc.get().toList + items should contain(s"Concurrent write to partition $i") + } + } + + it should "throw an exception when accessing an invalid partition index" in { + val invalidIndex = numOfPartitions + + val exception = intercept[RuntimeException] { + partitionDocument.getItem(invalidIndex) + } + + exception.getMessage should include(s"Index $invalidIndex out of bound") + } +} From e446e9c649cb7ca3718a4718ada0bf0ae2a6e473 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Tue, 10 Dec 2024 08:05:54 -0800 Subject: [PATCH 03/49] add more to unit tests --- .../result/ItemizedFileDocumentSpec.scala | 41 ++++++++++++++++++- .../result/PartitionDocumentSpec.scala | 5 +-- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala index 6e1c0f1ec72..c5400b42bc4 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala @@ -11,7 +11,6 @@ import org.scalatest.matchers.should.Matchers import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future -import scala.util.Using class ItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { @@ -102,6 +101,46 @@ class ItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndA .futureValue } + it should "handle concurrent reads and writes safely" in { + val numberOfWrites = 5 + val numberOfReads = 5 + + // Writer thread to add items + val writerFuture = Future { + fileDocument.open() + (1 to numberOfWrites).foreach { i => + fileDocument.putOne(s"Read-Write Test Write $i") + } + fileDocument.close() + } + + // Reader threads to read items concurrently + val readerFutures = (1 to numberOfReads).map { _ => + Future { + fileDocument.open() + val items = fileDocument.get().toList + fileDocument.close() + items + } + } + + // Wait for all futures to complete + val combinedFuture = for { + _ <- writerFuture + readerResults <- Future.sequence(readerFutures) + } yield readerResults + + val results = combinedFuture.futureValue + + // Verify the results + results.foreach { items => + items should contain(initialContent) + (1 to numberOfWrites).foreach { i => + items should contain(s"Read-Write Test Write $i") + } + } + } + it should "handle writing after reopening the file" in { fileDocument.open() fileDocument.putOne("First Write") diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala index d1bdbebc0cd..8df088e67bb 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala @@ -7,14 +7,13 @@ import org.scalatest.matchers.should.Matchers import java.net.URI import java.nio.file.{Files, Paths} -import scala.util.Using class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { var partitionDocument: PartitionDocument[String] = _ val numOfPartitions = 3 - val partitionId = - Files.createTempDirectory("partition_doc_test").toUri.toString + "/test_partition" + val partitionId: String = + Files.createTempDirectory("partition_doc_test").resolve("test_partition").toUri.toString before { // Initialize the PartitionDocument with a base ID and number of partitions From 9627b2578535a7ced9b07f5bbd5597b0ba00060d Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Wed, 11 Dec 2024 10:00:30 -0800 Subject: [PATCH 04/49] make PartitionDocument return T --- .../storage/result/ItemizedFileDocument.scala | 7 + .../storage/result/PartitionDocument.scala | 149 ++++++++++++------ .../result/PartitionDocumentSpec.scala | 56 ++++--- 3 files changed, 137 insertions(+), 75 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala index 0560c275137..5c6b370690f 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala @@ -155,6 +155,13 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) iterator.drop(i).next() } + override def getRange(from: Int, until: Int): Iterator[T] = get().slice(from, until) + + override def getAfter(offset: Int): Iterator[T] = get().drop(offset + 1) + + override def getCount: Long = get().size + + /** * Get an iterator of data items of type T. Each returned item will be deserialized using Kryo. * diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala index 5974dc645a3..6ba15b44f91 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala @@ -3,29 +3,28 @@ package edu.uci.ics.amber.core.storage.result import edu.uci.ics.amber.core.storage.model.VirtualDocument import java.net.URI +import java.util.concurrent.locks.ReentrantLock -/** - * PartitionDocument is a storage object that consists #numOfPartition physical files as its underlying data storage. - * Each underlying file's URI is in the format of {partitionDocumentURI}_{index}. - * - * PartitionDocument only support getting the FileDocument that corresponds to the single partition either by index or by iterator. - * To write over the partition, you should get the FileDocument first, then call write-related methods over it. FileDocument guarantees the thread-safe read/write. - * - * The Type parameter T is used to specify the type of data item stored in the partition - * @param id the id of this partition document. - * @param numOfPartition number of partitions - */ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) - extends VirtualDocument[ItemizedFileDocument[T]] { + extends VirtualDocument[T] { + + // Array of partitions + private val partitions = Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(i))) + + // Cursor for each partition to track read position + private val cursors = Array.fill(numOfPartition)(0) + + // Mutex for thread safety + private val mutex = new ReentrantLock() /** - * Utility functions to generate the partition URI by index - * @param i index of the partition - * @return the URI of the partition - */ + * Utility function to generate the partition URI by index. + * @param i Index of the partition. + * @return The URI of the partition. + */ private def getPartitionURI(i: Int): URI = { if (i < 0 || i >= numOfPartition) { - throw new RuntimeException(f"Index $i out of bound") + throw new RuntimeException(s"Index $i out of bounds") } new URI(s"${id}_partition$i") } @@ -36,43 +35,101 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: ) /** - * Get the partition by index i. - * This method is THREAD-UNSAFE, as multiple threads can get any partition by index. But the returned FileDocument is thread-safe - * @param i index starting from 0 - * @return FileDocument corresponds to the certain partition - */ - override def getItem(i: Int): ItemizedFileDocument[T] = { - new ItemizedFileDocument(getPartitionURI(i)) + * Get the partition item by index. + * This method uses a mutex to ensure thread safety when updating the cursor. + * @param i Index starting from 0. + * @return The data item of type T. + */ + override def getItem(i: Int): T = { + mutex.lock() + try { + val partitionIndex = i % numOfPartition + val document = partitions(partitionIndex) + val item = document.getItem(cursors(partitionIndex)) + cursors(partitionIndex) += 1 + item + } finally { + mutex.unlock() + } } /** - * Get the iterator of partitions. - * This method is THREAD-UNSAFE, as multiple threads can get the iterator and loop through all partitions. But the returned FileDocument is thread-safe - * @return an iterator that return the FileDocument corresponds to the certain partition - */ - override def get(): Iterator[ItemizedFileDocument[T]] = - new Iterator[ItemizedFileDocument[T]] { - private var i: Int = 0 - - override def hasNext: Boolean = i < numOfPartition - - override def next(): ItemizedFileDocument[T] = { - if (!hasNext) { - throw new NoSuchElementException("No more partitions") + * Get an iterator over all items in the partitions. + * @return An iterator that returns items of type T. + */ + override def get(): Iterator[T] = new Iterator[T] { + private var partitionIndex = 0 + private val iterators = partitions.map(_.get()) + + override def hasNext: Boolean = iterators.exists(_.hasNext) + + override def next(): T = { + mutex.lock() + try { + while (!iterators(partitionIndex).hasNext) { + partitionIndex = (partitionIndex + 1) % numOfPartition } - val document = new ItemizedFileDocument[T](getPartitionURI(i)) - i += 1 - document + iterators(partitionIndex).next() + } finally { + mutex.unlock() } } + } + + /** + * Get an iterator over a range of items. + * @param from The starting index (inclusive). + * @param until The ending index (exclusive). + * @return An iterator over the specified range of items. + */ + override def getRange(from: Int, until: Int): Iterator[T] = { + mutex.lock() + try { + get().slice(from, until) + } finally { + mutex.unlock() + } + } + + /** + * Get an iterator over all items after the specified index. + * @param offset The starting index (exclusive). + * @return An iterator over the items after the specified offset. + */ + override def getAfter(offset: Int): Iterator[T] = { + mutex.lock() + try { + get().drop(offset + 1) + } finally { + mutex.unlock() + } + } /** - * Remove all partitions. - * This method is THREAD-UNSAFE. But FileDocument's remove is thread-safe - */ + * Get the total count of items across all partitions. + * @return The total count of items. + */ + override def getCount: Long = { + mutex.lock() + try { + partitions.map(_.getCount).sum + } finally { + mutex.unlock() + } + } + + /** + * Remove all partitions. + * This method is thread-safe due to the mutex. + */ override def clear(): Unit = { - for (i <- 0 until numOfPartition) { - getItem(i).clear() + mutex.lock() + try { + for (partition <- partitions) { + partition.clear() + } + } finally { + mutex.unlock() } } -} +} \ No newline at end of file diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala index 8df088e67bb..aab324d4e22 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala @@ -12,8 +12,8 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte var partitionDocument: PartitionDocument[String] = _ val numOfPartitions = 3 - val partitionId: String = - Files.createTempDirectory("partition_doc_test").resolve("test_partition").toUri.toString + val tempDir = Files.createTempDirectory("partition_doc_test") + val partitionId: String = tempDir.resolve("test_partition").toUri.toString.stripSuffix("/") before { // Initialize the PartitionDocument with a base ID and number of partitions @@ -24,62 +24,60 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte // Clean up all partitions after each test partitionDocument.clear() for (i <- 0 until numOfPartitions) { - Files.deleteIfExists(Paths.get(new URI(s"${partitionId}_partition$i"))) + val partitionPath = Paths.get(new URI(s"${partitionId}_partition$i")) + Files.deleteIfExists(partitionPath) } - Files.deleteIfExists(Paths.get(new URI(partitionId).getPath.stripSuffix("/test_partition"))) + Files.deleteIfExists(tempDir) } - "PartitionDocument" should "create and write to each partition" in { + "PartitionDocument" should "create and write to each partition directly" in { for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) + val partitionURI = new URI(s"${partitionId}_partition$i") + val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Data for partition $i") fileDoc.close() } for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) - val items = fileDoc.get().toList - items should contain(s"Data for partition $i") + val item = partitionDocument.getItem(i) + item should be(s"Data for partition $i") } } it should "read from multiple partitions" in { - // Write some data to each partition + // Write some data directly to each partition for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) + val partitionURI = new URI(s"${partitionId}_partition$i") + val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Content in partition $i") fileDoc.close() } - // Read and verify data from each partition - val partitionIterator = partitionDocument.get() + // Read and verify data from each partition using PartitionDocument + val items = partitionDocument.get().toList for (i <- 0 until numOfPartitions) { - val fileDoc = partitionIterator.next() - val items = fileDoc.get().toList items should contain(s"Content in partition $i") } } it should "clear all partitions" in { - // Write some data to each partition + // Write some data directly to each partition for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) + val partitionURI = new URI(s"${partitionId}_partition$i") + val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Some data in partition $i") fileDoc.close() } - // Clear all partitions + // Clear all partitions using PartitionDocument partitionDocument.clear() // Verify that each partition is empty - for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) - val items = fileDoc.get().toList - items should be(empty) - } + val items = partitionDocument.get().toList + items should be(empty) } it should "handle concurrent writes to different partitions" in { @@ -88,7 +86,8 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte val futures = (0 until numOfPartitions).map { i => Future { - val fileDoc = partitionDocument.getItem(i) + val partitionURI = new URI(s"${partitionId}_partition$i") + val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Concurrent write to partition $i") fileDoc.close() @@ -97,10 +96,9 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte Future.sequence(futures).futureValue - // Verify data written concurrently + // Verify data written concurrently using PartitionDocument + val items = partitionDocument.get().toList for (i <- 0 until numOfPartitions) { - val fileDoc = partitionDocument.getItem(i) - val items = fileDoc.get().toList items should contain(s"Concurrent write to partition $i") } } @@ -112,6 +110,6 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte partitionDocument.getItem(invalidIndex) } - exception.getMessage should include(s"Index $invalidIndex out of bound") + exception.getMessage should include(s"Index $invalidIndex out of bounds") } -} +} \ No newline at end of file From b85fd45e2652a53ff9d5ee13fce150561ab99000 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Wed, 11 Dec 2024 10:45:17 -0800 Subject: [PATCH 05/49] fix partition document test --- .../storage/result/ItemizedFileDocument.scala | 30 +++--- .../storage/result/PartitionDocument.scala | 94 +++++++++---------- .../result/PartitionDocumentSpec.scala | 21 ++--- 3 files changed, 67 insertions(+), 78 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala index 5c6b370690f..d029e86b4d4 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala @@ -16,13 +16,10 @@ object ItemizedFileDocument { /** * ItemizedFileDocument provides methods to read/write items to a file located on the filesystem. - * All methods are THREAD-SAFE, implemented using a read-write lock: - * - 1 writer at a time: only 1 thread can acquire the write lock. - * - n readers at a time: multiple threads can acquire the read lock. - * * The type parameter T specifies the iterable data item stored in the file. * - * @param uri the identifier of the file. If the file doesn't physically exist, ItemizedFileDocument will create it during construction. + * @param uri the identifier of the file. + * If the file doesn't physically exist, ItemizedFileDocument will create it during construction. */ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) extends VirtualDocument[T] @@ -44,6 +41,17 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) } } + // Check and create the file if it does not exist + withWriteLock { + if (!file.exists()) { + val parentDir = file.getParent + if (parentDir != null && !parentDir.exists()) { + parentDir.createFolder() // Create all necessary parent directories + } + file.createFile() // Create the file if it does not exist + } + } + // Utility function to wrap code block with read lock private def withReadLock[M](block: => M): M = { lock.readLock().lock() @@ -64,17 +72,6 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) } } - // Check and create the file if it does not exist - withWriteLock { - if (!file.exists()) { - val parentDir = file.getParent - if (parentDir != null && !parentDir.exists()) { - parentDir.createFolder() // Create all necessary parent directories - } - file.createFile() // Create the file if it does not exist - } - } - /** * Append the content in the given object to the ItemizedFileDocument. This method is THREAD-SAFE. * Each record will be stored as . @@ -161,7 +158,6 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) override def getCount: Long = get().size - /** * Get an iterator of data items of type T. Each returned item will be deserialized using Kryo. * diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala index 6ba15b44f91..4467700ac42 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala @@ -6,10 +6,11 @@ import java.net.URI import java.util.concurrent.locks.ReentrantLock class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) - extends VirtualDocument[T] { + extends VirtualDocument[T] { // Array of partitions - private val partitions = Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(i))) + private val partitions = + Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(i))) // Cursor for each partition to track read position private val cursors = Array.fill(numOfPartition)(0) @@ -18,10 +19,10 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: private val mutex = new ReentrantLock() /** - * Utility function to generate the partition URI by index. - * @param i Index of the partition. - * @return The URI of the partition. - */ + * Utility function to generate the partition URI by index. + * @param i Index of the partition. + * @return The URI of the partition. + */ private def getPartitionURI(i: Int): URI = { if (i < 0 || i >= numOfPartition) { throw new RuntimeException(s"Index $i out of bounds") @@ -29,21 +30,23 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: new URI(s"${id}_partition$i") } + // use round-robin to decide which partition to go to + private def getPartitionIndex(i: Int): Int = i % numOfPartition + override def getURI: URI = throw new RuntimeException( "Partition Document doesn't physically exist. It is invalid to acquire its URI" ) /** - * Get the partition item by index. - * This method uses a mutex to ensure thread safety when updating the cursor. - * @param i Index starting from 0. - * @return The data item of type T. - */ + * Get the partition item by index. + * @param i Index starting from 0. + * @return The data item of type T. + */ override def getItem(i: Int): T = { mutex.lock() try { - val partitionIndex = i % numOfPartition + val partitionIndex = getPartitionIndex(i) val document = partitions(partitionIndex) val item = document.getItem(cursors(partitionIndex)) cursors(partitionIndex) += 1 @@ -53,35 +56,32 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: } } - /** - * Get an iterator over all items in the partitions. - * @return An iterator that returns items of type T. - */ - override def get(): Iterator[T] = new Iterator[T] { - private var partitionIndex = 0 - private val iterators = partitions.map(_.get()) - - override def hasNext: Boolean = iterators.exists(_.hasNext) - - override def next(): T = { - mutex.lock() - try { - while (!iterators(partitionIndex).hasNext) { - partitionIndex = (partitionIndex + 1) % numOfPartition + override def get(): Iterator[T] = + new Iterator[T] { + private var partitionIndex = 0 + private val iterators = partitions.map(_.get()) + + override def hasNext: Boolean = iterators.exists(_.hasNext) + + override def next(): T = { + mutex.lock() + try { + while (!iterators(partitionIndex).hasNext) { + partitionIndex = getPartitionIndex(partitionIndex + 1) + } + iterators(partitionIndex).next() + } finally { + mutex.unlock() } - iterators(partitionIndex).next() - } finally { - mutex.unlock() } } - } /** - * Get an iterator over a range of items. - * @param from The starting index (inclusive). - * @param until The ending index (exclusive). - * @return An iterator over the specified range of items. - */ + * Get an iterator over a range of items. + * @param from The starting index (inclusive). + * @param until The ending index (exclusive). + * @return An iterator over the specified range of items. + */ override def getRange(from: Int, until: Int): Iterator[T] = { mutex.lock() try { @@ -92,10 +92,10 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: } /** - * Get an iterator over all items after the specified index. - * @param offset The starting index (exclusive). - * @return An iterator over the items after the specified offset. - */ + * Get an iterator over all items after the specified index. + * @param offset The starting index (exclusive). + * @return An iterator over the items after the specified offset. + */ override def getAfter(offset: Int): Iterator[T] = { mutex.lock() try { @@ -106,9 +106,9 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: } /** - * Get the total count of items across all partitions. - * @return The total count of items. - */ + * Get the total count of items across all partitions. + * @return The total count of items. + */ override def getCount: Long = { mutex.lock() try { @@ -119,9 +119,9 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: } /** - * Remove all partitions. - * This method is thread-safe due to the mutex. - */ + * Remove all partitions. + * This method is thread-safe due to the mutex. + */ override def clear(): Unit = { mutex.lock() try { @@ -132,4 +132,4 @@ class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: mutex.unlock() } } -} \ No newline at end of file +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala index aab324d4e22..b8a1e239f37 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala @@ -1,5 +1,6 @@ package edu.uci.ics.amber.core.storage.result +import org.apache.commons.vfs2.{FileObject, VFS} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures.convertScalaFuture @@ -75,9 +76,11 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte // Clear all partitions using PartitionDocument partitionDocument.clear() - // Verify that each partition is empty - val items = partitionDocument.get().toList - items should be(empty) + for (i <- 0 until numOfPartitions) { + val partitionURI = new URI(s"${partitionId}_partition$i") + val file: FileObject = VFS.getManager.resolveFile(partitionURI.toString) + file.exists() should be(false) + } } it should "handle concurrent writes to different partitions" in { @@ -102,14 +105,4 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte items should contain(s"Concurrent write to partition $i") } } - - it should "throw an exception when accessing an invalid partition index" in { - val invalidIndex = numOfPartitions - - val exception = intercept[RuntimeException] { - partitionDocument.getItem(invalidIndex) - } - - exception.getMessage should include(s"Index $invalidIndex out of bounds") - } -} \ No newline at end of file +} From 8e6fec3d385b10203666c2818c5dfa9eda258daa Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Wed, 11 Dec 2024 16:07:53 -0800 Subject: [PATCH 06/49] refining the documents --- .../storage/result/ItemizedFileDocument.scala | 88 +++++++----- .../storage/result/PartitionDocument.scala | 135 ------------------ .../PartitionedItemizedFileDocument.scala | 111 ++++++++++++++ .../edu/uci/ics/amber/util/PathUtils.scala | 3 + ...PartitionedItemizedFileDocumentSpec.scala} | 28 ++-- 5 files changed, 175 insertions(+), 190 deletions(-) delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala rename core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/{PartitionDocumentSpec.scala => PartitionedItemizedFileDocumentSpec.scala} (73%) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala index d029e86b4d4..d5975c945f7 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala @@ -19,13 +19,15 @@ object ItemizedFileDocument { * The type parameter T specifies the iterable data item stored in the file. * * @param uri the identifier of the file. - * If the file doesn't physically exist, ItemizedFileDocument will create it during construction. + * If the file doesn't physically exist, ItemizedFileDocument will create the file(possibly also the parent folder) + * during its initialization. + * The lifecycle of the file is bundled with JVM, i.e. when JVM exits, the file gets deleted. */ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) extends VirtualDocument[T] with BufferedItemWriter[T] { - val file: FileObject = VFS.getManager.resolveFile(uri.toString) + val file: FileObject = VFS.getManager.resolveFile(uri) val lock = new ReentrantReadWriteLock() // Buffer to store items before flushing @@ -72,6 +74,37 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) } } + /** + * Utility function to get an iterator of data items of type T. + * Each returned item will be deserialized using Kryo. + */ + private def getIterator: Iterator[T] = { + lazy val input = new com.twitter.chill.Input(file.getContent.getInputStream) + new Iterator[T] { + var record: T = internalNext() + + private def internalNext(): T = { + try { + val len = input.readInt() + val bytes = input.readBytes(len) + ItemizedFileDocument.kryoPool.fromBytes(bytes).asInstanceOf[T] + } catch { + case _: Throwable => + input.close() + null + } + } + + override def next(): T = { + val currentRecord = record + record = internalNext() + currentRecord + } + + override def hasNext: Boolean = record != null + } + } + /** * Append the content in the given object to the ItemizedFileDocument. This method is THREAD-SAFE. * Each record will be stored as . @@ -147,47 +180,28 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) * @param i index starting from 0 * @return data item of type T */ - override def getItem(i: Int): T = { - val iterator = get() + override def getItem(i: Int): T = withReadLock { + val iterator = getIterator iterator.drop(i).next() } - override def getRange(from: Int, until: Int): Iterator[T] = get().slice(from, until) - - override def getAfter(offset: Int): Iterator[T] = get().drop(offset + 1) - - override def getCount: Long = get().size - - /** - * Get an iterator of data items of type T. Each returned item will be deserialized using Kryo. - * - * @return an iterator that returns data items of type T - */ - override def get(): Iterator[T] = { - lazy val input = new com.twitter.chill.Input(file.getContent.getInputStream) - new Iterator[T] { - var record: T = internalNext() + override def getRange(from: Int, until: Int): Iterator[T] = + withReadLock { + getIterator.slice(from, until) + } - private def internalNext(): T = { - try { - val len = input.readInt() - val bytes = input.readBytes(len) - ItemizedFileDocument.kryoPool.fromBytes(bytes).asInstanceOf[T] - } catch { - case _: Throwable => - input.close() - null - } - } + override def getAfter(offset: Int): Iterator[T] = + withReadLock { + getIterator.drop(offset + 1) + } - override def next(): T = { - val currentRecord = record - record = internalNext() - currentRecord - } + override def getCount: Long = + withReadLock { + getIterator.size + } - override def hasNext: Boolean = record != null - } + override def get(): Iterator[T] = withReadLock { + getIterator } /** diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala deleted file mode 100644 index 4467700ac42..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionDocument.scala +++ /dev/null @@ -1,135 +0,0 @@ -package edu.uci.ics.amber.core.storage.result - -import edu.uci.ics.amber.core.storage.model.VirtualDocument - -import java.net.URI -import java.util.concurrent.locks.ReentrantLock - -class PartitionDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) - extends VirtualDocument[T] { - - // Array of partitions - private val partitions = - Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(i))) - - // Cursor for each partition to track read position - private val cursors = Array.fill(numOfPartition)(0) - - // Mutex for thread safety - private val mutex = new ReentrantLock() - - /** - * Utility function to generate the partition URI by index. - * @param i Index of the partition. - * @return The URI of the partition. - */ - private def getPartitionURI(i: Int): URI = { - if (i < 0 || i >= numOfPartition) { - throw new RuntimeException(s"Index $i out of bounds") - } - new URI(s"${id}_partition$i") - } - - // use round-robin to decide which partition to go to - private def getPartitionIndex(i: Int): Int = i % numOfPartition - - override def getURI: URI = - throw new RuntimeException( - "Partition Document doesn't physically exist. It is invalid to acquire its URI" - ) - - /** - * Get the partition item by index. - * @param i Index starting from 0. - * @return The data item of type T. - */ - override def getItem(i: Int): T = { - mutex.lock() - try { - val partitionIndex = getPartitionIndex(i) - val document = partitions(partitionIndex) - val item = document.getItem(cursors(partitionIndex)) - cursors(partitionIndex) += 1 - item - } finally { - mutex.unlock() - } - } - - override def get(): Iterator[T] = - new Iterator[T] { - private var partitionIndex = 0 - private val iterators = partitions.map(_.get()) - - override def hasNext: Boolean = iterators.exists(_.hasNext) - - override def next(): T = { - mutex.lock() - try { - while (!iterators(partitionIndex).hasNext) { - partitionIndex = getPartitionIndex(partitionIndex + 1) - } - iterators(partitionIndex).next() - } finally { - mutex.unlock() - } - } - } - - /** - * Get an iterator over a range of items. - * @param from The starting index (inclusive). - * @param until The ending index (exclusive). - * @return An iterator over the specified range of items. - */ - override def getRange(from: Int, until: Int): Iterator[T] = { - mutex.lock() - try { - get().slice(from, until) - } finally { - mutex.unlock() - } - } - - /** - * Get an iterator over all items after the specified index. - * @param offset The starting index (exclusive). - * @return An iterator over the items after the specified offset. - */ - override def getAfter(offset: Int): Iterator[T] = { - mutex.lock() - try { - get().drop(offset + 1) - } finally { - mutex.unlock() - } - } - - /** - * Get the total count of items across all partitions. - * @return The total count of items. - */ - override def getCount: Long = { - mutex.lock() - try { - partitions.map(_.getCount).sum - } finally { - mutex.unlock() - } - } - - /** - * Remove all partitions. - * This method is thread-safe due to the mutex. - */ - override def clear(): Unit = { - mutex.lock() - try { - for (partition <- partitions) { - partition.clear() - } - } finally { - mutex.unlock() - } - } -} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala new file mode 100644 index 00000000000..0a80e8d45f5 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala @@ -0,0 +1,111 @@ + package edu.uci.ics.amber.core.storage.result + + import edu.uci.ics.amber.core.storage.model.VirtualDocument + import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI + import edu.uci.ics.amber.util.PathUtils.workflowResultsRootPath + import org.apache.commons.vfs2.VFS + + import java.net.URI + import java.util.concurrent.locks.ReentrantLock + + + object PartitionedItemizedFileDocument { + + /** + * Utility function to generate the partition URI by index. + */ + def getPartitionURI(id: String, i: Int): URI = { + workflowResultsRootPath.resolve(id).resolve(s"partition$i").toUri + } + } + + class PartitionedItemizedFileDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) + extends VirtualDocument[T] { + + // The array of itemized file documents, each stands for a partition + private val partitions = + Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(id, i))) + + // Cursor for each partition to track read position + private val cursors = Array.fill(numOfPartition)(0) + + private val mutex = new ReentrantLock() + + // use round-robin to decide which partition to go to when reading i-th item + private def getPartitionIndex(i: Int): Int = i % numOfPartition + + override def getURI: URI = + workflowResultsRootPath.resolve(id).toUri + + override def getItem(i: Int): T = { + mutex.lock() + try { + val partitionIndex = getPartitionIndex(i) + val document = partitions(partitionIndex) + val item = document.getItem(cursors(partitionIndex)) + cursors(partitionIndex) += 1 + item + } finally { + mutex.unlock() + } + } + + override def get(): Iterator[T] = + new Iterator[T] { + private var partitionIndex = 0 + private val iterators = partitions.map(_.get()) + + override def hasNext: Boolean = iterators.exists(_.hasNext) + + override def next(): T = { + mutex.lock() + try { + while (!iterators(partitionIndex).hasNext) { + partitionIndex = getPartitionIndex(partitionIndex + 1) + } + iterators(partitionIndex).next() + } finally { + mutex.unlock() + } + } + } + + override def getRange(from: Int, until: Int): Iterator[T] = { + mutex.lock() + try { + get().slice(from, until) + } finally { + mutex.unlock() + } + } + + override def getAfter(offset: Int): Iterator[T] = { + mutex.lock() + try { + get().drop(offset + 1) + } finally { + mutex.unlock() + } + } + + override def getCount: Long = { + mutex.lock() + try { + partitions.map(_.getCount).sum + } finally { + mutex.unlock() + } + } + + override def clear(): Unit = { + mutex.lock() + try { + for (partition <- partitions) { + partition.clear() + } + VFS.getManager.resolveFile(getURI).delete() + } finally { + mutex.unlock() + } + } + } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala index 7fe89d0e8ce..1a18e59aad9 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala @@ -39,6 +39,9 @@ object PathUtils { lazy val workflowCompilingServicePath: Path = corePath.resolve("workflow-compiling-service") + lazy val workflowResultsRootPath: Path = + corePath.resolve("amber").resolve("user-resources").resolve("workflow-results") + private lazy val datasetsRootPath = corePath.resolve("amber").resolve("user-resources").resolve("datasets") diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala similarity index 73% rename from core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala rename to core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala index b8a1e239f37..7f113420609 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala @@ -1,39 +1,31 @@ package edu.uci.ics.amber.core.storage.result +import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI import org.apache.commons.vfs2.{FileObject, VFS} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures.convertScalaFuture import org.scalatest.matchers.should.Matchers -import java.net.URI -import java.nio.file.{Files, Paths} +class PartitionedItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { -class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - - var partitionDocument: PartitionDocument[String] = _ + var partitionDocument: PartitionedItemizedFileDocument[String] = _ val numOfPartitions = 3 - val tempDir = Files.createTempDirectory("partition_doc_test") - val partitionId: String = tempDir.resolve("test_partition").toUri.toString.stripSuffix("/") + val partitionId: String = "partition_doc_test" before { // Initialize the PartitionDocument with a base ID and number of partitions - partitionDocument = new PartitionDocument[String](partitionId, numOfPartitions) + partitionDocument = new PartitionedItemizedFileDocument[String](partitionId, numOfPartitions) } after { // Clean up all partitions after each test partitionDocument.clear() - for (i <- 0 until numOfPartitions) { - val partitionPath = Paths.get(new URI(s"${partitionId}_partition$i")) - Files.deleteIfExists(partitionPath) - } - Files.deleteIfExists(tempDir) } "PartitionDocument" should "create and write to each partition directly" in { for (i <- 0 until numOfPartitions) { - val partitionURI = new URI(s"${partitionId}_partition$i") + val partitionURI = getPartitionURI(partitionId, i) val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Data for partition $i") @@ -49,7 +41,7 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte it should "read from multiple partitions" in { // Write some data directly to each partition for (i <- 0 until numOfPartitions) { - val partitionURI = new URI(s"${partitionId}_partition$i") + val partitionURI = getPartitionURI(partitionId, i) val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Content in partition $i") @@ -66,7 +58,7 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte it should "clear all partitions" in { // Write some data directly to each partition for (i <- 0 until numOfPartitions) { - val partitionURI = new URI(s"${partitionId}_partition$i") + val partitionURI = getPartitionURI(partitionId, i) val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Some data in partition $i") @@ -77,7 +69,7 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte partitionDocument.clear() for (i <- 0 until numOfPartitions) { - val partitionURI = new URI(s"${partitionId}_partition$i") + val partitionURI = getPartitionURI(partitionId, i) val file: FileObject = VFS.getManager.resolveFile(partitionURI.toString) file.exists() should be(false) } @@ -89,7 +81,7 @@ class PartitionDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte val futures = (0 until numOfPartitions).map { i => Future { - val partitionURI = new URI(s"${partitionId}_partition$i") + val partitionURI = getPartitionURI(partitionId, i) val fileDoc = new ItemizedFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Concurrent write to partition $i") From 288aea4448d0181a6af8c395256590f38978a59f Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 12 Dec 2024 08:46:22 -0800 Subject: [PATCH 07/49] add type R to PartitionedItemizedFileDocument --- .../storage/result/ItemizedFileDocument.scala | 26 +-- .../PartitionedItemizedFileDocument.scala | 185 +++++++++--------- .../PartitionedItemizedFileDocumentSpec.scala | 8 +- 3 files changed, 116 insertions(+), 103 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala index d5975c945f7..089b5a3cb5d 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala @@ -20,7 +20,7 @@ object ItemizedFileDocument { * * @param uri the identifier of the file. * If the file doesn't physically exist, ItemizedFileDocument will create the file(possibly also the parent folder) - * during its initialization. + * during its initialization. * The lifecycle of the file is bundled with JVM, i.e. when JVM exits, the file gets deleted. */ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) @@ -75,9 +75,9 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) } /** - * Utility function to get an iterator of data items of type T. - * Each returned item will be deserialized using Kryo. - */ + * Utility function to get an iterator of data items of type T. + * Each returned item will be deserialized using Kryo. + */ private def getIterator: Iterator[T] = { lazy val input = new com.twitter.chill.Input(file.getContent.getInputStream) new Iterator[T] { @@ -180,10 +180,11 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) * @param i index starting from 0 * @return data item of type T */ - override def getItem(i: Int): T = withReadLock { - val iterator = getIterator - iterator.drop(i).next() - } + override def getItem(i: Int): T = + withReadLock { + val iterator = getIterator + iterator.drop(i).next() + } override def getRange(from: Int, until: Int): Iterator[T] = withReadLock { @@ -198,11 +199,12 @@ class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) override def getCount: Long = withReadLock { getIterator.size - } + } - override def get(): Iterator[T] = withReadLock { - getIterator - } + override def get(): Iterator[T] = + withReadLock { + getIterator + } /** * Physically remove the file specified by the URI. This method is THREAD-SAFE. diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala index 0a80e8d45f5..ab0ac3dfcfd 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala @@ -1,111 +1,118 @@ - package edu.uci.ics.amber.core.storage.result +package edu.uci.ics.amber.core.storage.result - import edu.uci.ics.amber.core.storage.model.VirtualDocument - import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI - import edu.uci.ics.amber.util.PathUtils.workflowResultsRootPath - import org.apache.commons.vfs2.VFS +import edu.uci.ics.amber.core.storage.model.VirtualDocument +import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI +import edu.uci.ics.amber.util.PathUtils.workflowResultsRootPath +import org.apache.commons.vfs2.VFS - import java.net.URI - import java.util.concurrent.locks.ReentrantLock +import java.net.URI +import java.util.concurrent.locks.ReentrantLock +object PartitionedItemizedFileDocument { - object PartitionedItemizedFileDocument { - - /** - * Utility function to generate the partition URI by index. - */ - def getPartitionURI(id: String, i: Int): URI = { - workflowResultsRootPath.resolve(id).resolve(s"partition$i").toUri + /** + * Utility function to generate the partition URI by index. + */ + def getPartitionURI(id: String, i: Int): URI = { + workflowResultsRootPath.resolve(id).resolve(s"partition$i").toUri + } +} + +class PartitionedItemizedFileDocument[R <: VirtualDocument[T], T >: Null <: AnyRef]( + val id: String, + val numOfPartition: Int, + createPartition: URI => R +) extends VirtualDocument[T] { + + // The vector of partitions, each being an instance of R (a subclass of VirtualDocument[T]) + private val partitions = + Vector.tabulate(numOfPartition)(i => createPartition(getPartitionURI(id, i))) + + // Cursor for each partition to track read position + private val cursors = Array.fill(numOfPartition)(0) + + private val mutex = new ReentrantLock() + + // Use round-robin to decide which partition to go to when reading the i-th item + private def getPartitionIndex(i: Int): Int = i % numOfPartition + + override def getURI: URI = workflowResultsRootPath.resolve(id).toUri + + override def getItem(i: Int): T = { + mutex.lock() + try { + val partitionIndex = getPartitionIndex(i) + val document = partitions(partitionIndex) + val item = document.getItem(cursors(partitionIndex)) + cursors(partitionIndex) += 1 + item + } finally { + mutex.unlock() } } - class PartitionedItemizedFileDocument[T >: Null <: AnyRef](val id: String, val numOfPartition: Int) - extends VirtualDocument[T] { - - // The array of itemized file documents, each stands for a partition - private val partitions = - Array.tabulate(numOfPartition)(i => new ItemizedFileDocument[T](getPartitionURI(id, i))) - - // Cursor for each partition to track read position - private val cursors = Array.fill(numOfPartition)(0) - - private val mutex = new ReentrantLock() + override def get(): Iterator[T] = + new Iterator[T] { + private var partitionIndex = 0 + private val iterators = partitions.map(_.get()) - // use round-robin to decide which partition to go to when reading i-th item - private def getPartitionIndex(i: Int): Int = i % numOfPartition + override def hasNext: Boolean = iterators.exists(_.hasNext) - override def getURI: URI = - workflowResultsRootPath.resolve(id).toUri - - override def getItem(i: Int): T = { - mutex.lock() - try { - val partitionIndex = getPartitionIndex(i) - val document = partitions(partitionIndex) - val item = document.getItem(cursors(partitionIndex)) - cursors(partitionIndex) += 1 - item - } finally { - mutex.unlock() - } - } - - override def get(): Iterator[T] = - new Iterator[T] { - private var partitionIndex = 0 - private val iterators = partitions.map(_.get()) - - override def hasNext: Boolean = iterators.exists(_.hasNext) - - override def next(): T = { - mutex.lock() - try { - while (!iterators(partitionIndex).hasNext) { - partitionIndex = getPartitionIndex(partitionIndex + 1) - } - iterators(partitionIndex).next() - } finally { - mutex.unlock() + override def next(): T = { + mutex.lock() + try { + while (!iterators(partitionIndex).hasNext) { + partitionIndex = getPartitionIndex(partitionIndex + 1) } + iterators(partitionIndex).next() + } finally { + mutex.unlock() } } + } - override def getRange(from: Int, until: Int): Iterator[T] = { - mutex.lock() - try { - get().slice(from, until) - } finally { - mutex.unlock() - } + override def getRange(from: Int, until: Int): Iterator[T] = { + mutex.lock() + try { + get().slice(from, until) + } finally { + mutex.unlock() } + } - override def getAfter(offset: Int): Iterator[T] = { - mutex.lock() - try { - get().drop(offset + 1) - } finally { - mutex.unlock() - } + override def getAfter(offset: Int): Iterator[T] = { + mutex.lock() + try { + get().drop(offset + 1) + } finally { + mutex.unlock() } + } - override def getCount: Long = { - mutex.lock() - try { - partitions.map(_.getCount).sum - } finally { - mutex.unlock() - } + override def getCount: Long = { + mutex.lock() + try { + partitions.map(_.getCount).sum + } finally { + mutex.unlock() } + } - override def clear(): Unit = { - mutex.lock() - try { - for (partition <- partitions) { - partition.clear() - } - VFS.getManager.resolveFile(getURI).delete() - } finally { - mutex.unlock() + override def clear(): Unit = { + mutex.lock() + try { + // Clear each partition first + for (partition <- partitions) { + partition.clear() + } + + // Delete the directory containing all partitions + val directory = VFS.getManager.resolveFile(getURI) + if (directory.exists()) { + directory.delete() // Deletes the directory and its contents } + } finally { + mutex.unlock() } } +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala index 7f113420609..1c204341e73 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala @@ -9,13 +9,17 @@ import org.scalatest.matchers.should.Matchers class PartitionedItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - var partitionDocument: PartitionedItemizedFileDocument[String] = _ + var partitionDocument: PartitionedItemizedFileDocument[ItemizedFileDocument[String], String] = _ val numOfPartitions = 3 val partitionId: String = "partition_doc_test" before { // Initialize the PartitionDocument with a base ID and number of partitions - partitionDocument = new PartitionedItemizedFileDocument[String](partitionId, numOfPartitions) + partitionDocument = new PartitionedItemizedFileDocument[ItemizedFileDocument[String], String]( + partitionId, + numOfPartitions, + uri => new ItemizedFileDocument[String](uri) + ) } after { From c3a1d0034ae33660dd9ebc3b5e4f1d37db308d12 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 12 Dec 2024 08:47:19 -0800 Subject: [PATCH 08/49] do a rename --- ...edFileDocument.scala => PartitionedFileDocument.scala} | 6 +++--- ...cumentSpec.scala => PartitionedFileDocumentSpec.scala} | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) rename core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/{PartitionedItemizedFileDocument.scala => PartitionedFileDocument.scala} (93%) rename core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/{PartitionedItemizedFileDocumentSpec.scala => PartitionedFileDocumentSpec.scala} (88%) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala similarity index 93% rename from core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala rename to core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala index ab0ac3dfcfd..fbb67703f02 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedItemizedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala @@ -1,14 +1,14 @@ package edu.uci.ics.amber.core.storage.result import edu.uci.ics.amber.core.storage.model.VirtualDocument -import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI +import edu.uci.ics.amber.core.storage.result.PartitionedFileDocument.getPartitionURI import edu.uci.ics.amber.util.PathUtils.workflowResultsRootPath import org.apache.commons.vfs2.VFS import java.net.URI import java.util.concurrent.locks.ReentrantLock -object PartitionedItemizedFileDocument { +object PartitionedFileDocument { /** * Utility function to generate the partition URI by index. @@ -18,7 +18,7 @@ object PartitionedItemizedFileDocument { } } -class PartitionedItemizedFileDocument[R <: VirtualDocument[T], T >: Null <: AnyRef]( +class PartitionedFileDocument[R <: VirtualDocument[T], T >: Null <: AnyRef]( val id: String, val numOfPartition: Int, createPartition: URI => R diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala similarity index 88% rename from core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala rename to core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala index 1c204341e73..4a3e6bee47e 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedItemizedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala @@ -1,21 +1,21 @@ package edu.uci.ics.amber.core.storage.result -import edu.uci.ics.amber.core.storage.result.PartitionedItemizedFileDocument.getPartitionURI +import edu.uci.ics.amber.core.storage.result.PartitionedFileDocument.getPartitionURI import org.apache.commons.vfs2.{FileObject, VFS} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures.convertScalaFuture import org.scalatest.matchers.should.Matchers -class PartitionedItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { +class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - var partitionDocument: PartitionedItemizedFileDocument[ItemizedFileDocument[String], String] = _ + var partitionDocument: PartitionedFileDocument[ItemizedFileDocument[String], String] = _ val numOfPartitions = 3 val partitionId: String = "partition_doc_test" before { // Initialize the PartitionDocument with a base ID and number of partitions - partitionDocument = new PartitionedItemizedFileDocument[ItemizedFileDocument[String], String]( + partitionDocument = new PartitionedFileDocument[ItemizedFileDocument[String], String]( partitionId, numOfPartitions, uri => new ItemizedFileDocument[String](uri) From 97c601ea9649f39761096167541351d87878bcd0 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 12 Dec 2024 14:09:43 -0800 Subject: [PATCH 09/49] adding the arrow file document, TODO: fix the test --- .../storage/result/ArrowFileDocument.scala | 165 +++++++++++++ .../storage/result/ItemizedFileDocument.scala | 221 ------------------ .../result/ArrowFileDocumentSpec.scala | 127 ++++++++++ .../result/ItemizedFileDocumentSpec.scala | 160 ------------- .../result/PartitionedFileDocumentSpec.scala | 14 +- 5 files changed, 299 insertions(+), 388 deletions(-) create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala delete mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala new file mode 100644 index 00000000000..dc33304bbc9 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala @@ -0,0 +1,165 @@ +package edu.uci.ics.amber.core.storage.result + +import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.ipc.{ArrowFileReader, ArrowFileWriter} +import org.apache.arrow.vector.{VectorSchemaRoot, FieldVector} +import org.apache.arrow.vector.types.pojo.Schema +import org.apache.commons.vfs2.{FileObject, VFS} + +import java.io.{FileInputStream, FileOutputStream, DataOutputStream} +import java.net.URI +import java.nio.channels.{FileChannel, SeekableByteChannel} +import java.nio.file.{Paths, StandardOpenOption} +import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.collection.mutable.ArrayBuffer +import scala.util.Using + +class ArrowFileDocument[T]( + val uri: URI, + val arrowSchema: Schema, + val serializer: (T, Int, VectorSchemaRoot) => Unit, + val deserializer: (Int, VectorSchemaRoot) => T + ) extends VirtualDocument[T] with BufferedItemWriter[T] { + + private val file: FileObject = VFS.getManager.resolveFile(uri) + private val lock = new ReentrantReadWriteLock() + private val allocator = new RootAllocator() + private val buffer = new ArrayBuffer[T]() + override val bufferSize: Int = 1024 + + // Initialize the file if it doesn't exist + withWriteLock { + if (!file.exists()) { + val parentDir = file.getParent + if (parentDir != null && !parentDir.exists()) { + parentDir.createFolder() + } + file.createFile() + } + } + + // Utility function to wrap code block with read lock + private def withReadLock[M](block: => M): M = { + lock.readLock().lock() + try block + finally lock.readLock().unlock() + } + + // Utility function to wrap code block with write lock + private def withWriteLock[M](block: => M): M = { + lock.writeLock().lock() + try block + finally lock.writeLock().unlock() + } + + override def putOne(item: T): Unit = withWriteLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() + } + } + + override def removeOne(item: T): Unit = withWriteLock { + buffer -= item + } + + /** Write buffered items to the file and clear the buffer */ + private def flushBuffer(): Unit = withWriteLock { + val outputStream = new FileOutputStream(file.getURL.getPath, true) + Using.Manager { use => + val root = VectorSchemaRoot.create(arrowSchema, allocator) + val writer = new ArrowFileWriter(root, null, outputStream.getChannel) + use(writer) + use(root) + + writer.start() + + buffer.zipWithIndex.foreach { case (item, index) => + serializer(item, index, root) + } + + root.setRowCount(buffer.size) + writer.writeBatch() + buffer.clear() + writer.end() + } + } + + /** Open the writer (clear the buffer) */ + override def open(): Unit = withWriteLock { + buffer.clear() + } + + /** Close the writer, flushing any remaining buffered items */ + override def close(): Unit = withWriteLock { + if (buffer.nonEmpty) { + flushBuffer() + } + allocator.close() + } + + /** Get an iterator of data items of type T */ + private def getIterator: Iterator[T] = withReadLock { + val path = Paths.get(file.getURL.toURI) + val channel: SeekableByteChannel = FileChannel.open(path, StandardOpenOption.READ) + val reader = new ArrowFileReader(channel, allocator) + val root = reader.getVectorSchemaRoot + + new Iterator[T] { + private var currentIndex = 0 + private var currentBatchLoaded = reader.loadNextBatch() + + private def loadNextBatch(): Boolean = { + currentBatchLoaded = reader.loadNextBatch() + currentIndex = 0 + currentBatchLoaded + } + + override def hasNext: Boolean = currentIndex < root.getRowCount || loadNextBatch() + + override def next(): T = { + if (!hasNext) throw new NoSuchElementException("No more elements") + val item = deserializer(currentIndex, root) + currentIndex += 1 + item + } + } + } + + /** Get the ith data item */ + override def getItem(i: Int): T = withReadLock { + getIterator.drop(i).next() + } + + /** Get a range of data items */ + override def getRange(from: Int, until: Int): Iterator[T] = withReadLock { + getIterator.slice(from, until) + } + + /** Get items after a certain offset */ + override def getAfter(offset: Int): Iterator[T] = withReadLock { + getIterator.drop(offset + 1) + } + + /** Get the total count of items */ + override def getCount: Long = withReadLock { + getIterator.size + } + + /** Get all items as an iterator */ + override def get(): Iterator[T] = withReadLock { + getIterator + } + + /** Physically remove the file */ + override def clear(): Unit = withWriteLock { + if (file.exists()) { + file.delete() + } else { + throw new RuntimeException(s"File $uri doesn't exist") + } + } + + override def getURI: URI = uri +} \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala deleted file mode 100644 index 089b5a3cb5d..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ItemizedFileDocument.scala +++ /dev/null @@ -1,221 +0,0 @@ -package edu.uci.ics.amber.core.storage.result - -import com.twitter.chill.{KryoPool, ScalaKryoInstantiator} -import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} -import org.apache.commons.vfs2.{FileObject, VFS} - -import java.io.{DataOutputStream, InputStream} -import java.net.URI -import java.util.concurrent.locks.ReentrantReadWriteLock -import scala.collection.mutable.ArrayBuffer - -object ItemizedFileDocument { - // Initialize KryoPool as a static object - private val kryoPool = KryoPool.withByteArrayOutputStream(10, new ScalaKryoInstantiator) -} - -/** - * ItemizedFileDocument provides methods to read/write items to a file located on the filesystem. - * The type parameter T specifies the iterable data item stored in the file. - * - * @param uri the identifier of the file. - * If the file doesn't physically exist, ItemizedFileDocument will create the file(possibly also the parent folder) - * during its initialization. - * The lifecycle of the file is bundled with JVM, i.e. when JVM exits, the file gets deleted. - */ -class ItemizedFileDocument[T >: Null <: AnyRef](val uri: URI) - extends VirtualDocument[T] - with BufferedItemWriter[T] { - - val file: FileObject = VFS.getManager.resolveFile(uri) - val lock = new ReentrantReadWriteLock() - - // Buffer to store items before flushing - private val buffer = new ArrayBuffer[T]() - override val bufferSize: Int = 1024 - - // Register a shutdown hook to delete the file when the JVM exits - sys.addShutdownHook { - withWriteLock { - if (file.exists()) { - file.delete() - } - } - } - - // Check and create the file if it does not exist - withWriteLock { - if (!file.exists()) { - val parentDir = file.getParent - if (parentDir != null && !parentDir.exists()) { - parentDir.createFolder() // Create all necessary parent directories - } - file.createFile() // Create the file if it does not exist - } - } - - // Utility function to wrap code block with read lock - private def withReadLock[M](block: => M): M = { - lock.readLock().lock() - try { - block - } finally { - lock.readLock().unlock() - } - } - - // Utility function to wrap code block with write lock - private def withWriteLock[M](block: => M): M = { - lock.writeLock().lock() - try { - block - } finally { - lock.writeLock().unlock() - } - } - - /** - * Utility function to get an iterator of data items of type T. - * Each returned item will be deserialized using Kryo. - */ - private def getIterator: Iterator[T] = { - lazy val input = new com.twitter.chill.Input(file.getContent.getInputStream) - new Iterator[T] { - var record: T = internalNext() - - private def internalNext(): T = { - try { - val len = input.readInt() - val bytes = input.readBytes(len) - ItemizedFileDocument.kryoPool.fromBytes(bytes).asInstanceOf[T] - } catch { - case _: Throwable => - input.close() - null - } - } - - override def next(): T = { - val currentRecord = record - record = internalNext() - currentRecord - } - - override def hasNext: Boolean = record != null - } - } - - /** - * Append the content in the given object to the ItemizedFileDocument. This method is THREAD-SAFE. - * Each record will be stored as . - * - * @param item the content to append - */ - override def append(item: T): Unit = - withWriteLock { - buffer.append(item) - if (buffer.size >= bufferSize) { - flushBuffer() - } - } - - /** - * Write buffered items to the file and clear the buffer. - */ - private def flushBuffer(): Unit = - withWriteLock { - val outStream = file.getContent.getOutputStream(true) - val dataOutStream = new DataOutputStream(outStream) - try { - buffer.foreach { item => - val serializedBytes = ItemizedFileDocument.kryoPool.toBytesWithClass(item) - dataOutStream.writeInt(serializedBytes.length) - dataOutStream.write(serializedBytes) - } - buffer.clear() - } finally { - dataOutStream.close() - outStream.close() - } - } - - /** - * Open the writer. Initializes the buffer. - */ - override def open(): Unit = - withWriteLock { - buffer.clear() - } - - /** - * Close the writer, flushing any remaining buffered items to the file. - */ - override def close(): Unit = - withWriteLock { - if (buffer.nonEmpty) { - flushBuffer() - } - } - - /** - * Put one item into the buffer. Flushes if the buffer is full. - * - * @param item the data item to be written - */ - override def putOne(item: T): Unit = append(item) - - /** - * Remove one item from the buffer. This does not affect items already written to the file. - * - * @param item the item to remove - */ - override def removeOne(item: T): Unit = - withWriteLock { - buffer -= item - } - - /** - * Get the ith data item. The returned value will be deserialized using Kryo. - * - * @param i index starting from 0 - * @return data item of type T - */ - override def getItem(i: Int): T = - withReadLock { - val iterator = getIterator - iterator.drop(i).next() - } - - override def getRange(from: Int, until: Int): Iterator[T] = - withReadLock { - getIterator.slice(from, until) - } - - override def getAfter(offset: Int): Iterator[T] = - withReadLock { - getIterator.drop(offset + 1) - } - - override def getCount: Long = - withReadLock { - getIterator.size - } - - override def get(): Iterator[T] = - withReadLock { - getIterator - } - - /** - * Physically remove the file specified by the URI. This method is THREAD-SAFE. - */ - override def clear(): Unit = - withWriteLock { - if (!file.exists()) { - throw new RuntimeException(s"File $uri doesn't exist") - } - file.delete() - } - - override def getURI: URI = uri -} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala new file mode 100644 index 00000000000..e8185e1a380 --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala @@ -0,0 +1,127 @@ +package edu.uci.ics.amber.storage.result + +import edu.uci.ics.amber.core.storage.result.ArrowFileDocument +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.arrow.vector.{IntVector, VarCharVector, VectorSchemaRoot} +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.ScalaFutures.convertScalaFuture +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + +import java.net.URI +import java.nio.file.{Files, Paths} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future +import scala.jdk.CollectionConverters._ + +class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { + + val stringArrowSchema = new Schema(List( + Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) + ).asJava) + + def stringSerializer(item: String, index: Int, root: VectorSchemaRoot): Unit = { + val vector = root.getVector("data").asInstanceOf[VarCharVector] + vector.setSafe(index, item.getBytes("UTF-8")) + } + + def stringDeserializer(index: Int, root: VectorSchemaRoot): String = { + new String(root.getVector("data").asInstanceOf[VarCharVector].get(index)) + } + + def createDocument(): ArrowFileDocument[String] = { + val tempPath = Files.createTempFile("arrow_test", ".arrow") + new ArrowFileDocument[String](tempPath.toUri, stringArrowSchema, stringSerializer, stringDeserializer) + } + + def deleteDocument(doc: ArrowFileDocument[String]): Unit = { + Files.deleteIfExists(Paths.get(doc.getURI)) + } + + "ArrowFileDocument" should "allow writing and flushing buffered items" in { + val doc = createDocument() + doc.open() + doc.putOne("Buffered Item 1") + doc.putOne("Buffered Item 2") + doc.close() + + val items = doc.get().toList + items should contain("Buffered Item 1") + items should contain("Buffered Item 2") + + deleteDocument(doc) + } + + it should "correctly flush the buffer when it reaches the buffer size" in { + val doc = createDocument() + val largeBuffer = (1 to doc.bufferSize).map(i => s"Item $i") + + doc.open() + largeBuffer.foreach(item => doc.putOne(item)) + doc.close() + + val items = doc.get().toList + largeBuffer.foreach { item => + items should contain(item) + } + + deleteDocument(doc) + } + + it should "allow removing items from the buffer" in { + val doc = createDocument() + doc.open() + doc.putOne("Item to keep") + doc.putOne("Item to remove") + doc.removeOne("Item to remove") + doc.close() + + val items = doc.get().toList + items should contain("Item to keep") + items should not contain "Item to remove" + + deleteDocument(doc) + } + + it should "handle concurrent buffered writes safely" in { + val doc = createDocument() + val numberOfThreads = 5 + + val futures = (1 to numberOfThreads).map { i => + Future { + doc.open() + doc.putOne(s"Content from thread $i") + doc.close() + } + } + + Future.sequence(futures).futureValue + + val items = doc.get().toList + (1 to numberOfThreads).foreach { i => + items should contain(s"Content from thread $i") + } + + deleteDocument(doc) + } + + it should "handle writing after reopening the file" in { + val doc = createDocument() + + // First write + doc.open() + doc.putOne("First Write") + doc.close() + + // Second write + doc.open() + doc.putOne("Second Write") + doc.close() + + val items = doc.get().toList + items should contain("First Write") + items should contain("Second Write") + + deleteDocument(doc) + } +} \ No newline at end of file diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala deleted file mode 100644 index c5400b42bc4..00000000000 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ItemizedFileDocumentSpec.scala +++ /dev/null @@ -1,160 +0,0 @@ -package edu.uci.ics.amber.storage.result - -import edu.uci.ics.amber.core.storage.result.ItemizedFileDocument - -import java.net.URI -import java.nio.file.{Files, Paths} -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.ScalaFutures.convertScalaFuture -import org.scalatest.matchers.should.Matchers - -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.Future - -class ItemizedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - - var tempFileURI: URI = _ - var fileDocument: ItemizedFileDocument[String] = _ - - val initialContent = "Initial Content" - val newContent = "New Content" - - before { - // Generate a path for a temporary file - val tempPath = Files.createTempFile("", "") - tempFileURI = tempPath.toUri - fileDocument = new ItemizedFileDocument(tempFileURI) - - // Write initial content to file - fileDocument.putOne(initialContent) - fileDocument.close() - } - - after { - // Delete the temporary file - Files.deleteIfExists(Paths.get(tempFileURI)) - } - - "ItemizedFileDocument" should "allow writing and flushing buffered items" in { - // Add items to the buffer - fileDocument.open() - fileDocument.putOne("Buffered Item 1") - fileDocument.putOne("Buffered Item 2") - - // Force a flush - fileDocument.close() - - // Verify the items using the itemized get method - fileDocument.getItem(0) should equal(initialContent) - fileDocument.getItem(1) should equal("Buffered Item 1") - fileDocument.getItem(2) should equal("Buffered Item 2") - } - - it should "correctly flush the buffer when it reaches the buffer size" in { - val largeBuffer = (1 to fileDocument.bufferSize).map(i => s"Item $i") - - fileDocument.open() - largeBuffer.foreach(item => fileDocument.putOne(item)) - fileDocument.close() - - val items = fileDocument.get().toList - - items should contain(initialContent) - largeBuffer.foreach { item => - items should contain(item) - } - } - - it should "allow removing items from the buffer" in { - fileDocument.open() - fileDocument.putOne("Item to keep") - fileDocument.putOne("Item to remove") - fileDocument.removeOne("Item to remove") - fileDocument.close() - - val items = fileDocument.get().toList - - items should contain("Item to keep") - items should not contain "Item to remove" - } - - it should "handle concurrent buffered writes safely" in { - val numberOfThreads = 5 - val futures = (1 to numberOfThreads).map { i => - Future { - fileDocument.putOne(s"Content from thread $i") - } - } - - Future - .sequence(futures) - .map { _ => - fileDocument.close() - val items = fileDocument.get().toList - - items should contain(initialContent) - (1 to numberOfThreads).foreach { i => - items should contain(s"Content from thread $i") - } - } - .futureValue - } - - it should "handle concurrent reads and writes safely" in { - val numberOfWrites = 5 - val numberOfReads = 5 - - // Writer thread to add items - val writerFuture = Future { - fileDocument.open() - (1 to numberOfWrites).foreach { i => - fileDocument.putOne(s"Read-Write Test Write $i") - } - fileDocument.close() - } - - // Reader threads to read items concurrently - val readerFutures = (1 to numberOfReads).map { _ => - Future { - fileDocument.open() - val items = fileDocument.get().toList - fileDocument.close() - items - } - } - - // Wait for all futures to complete - val combinedFuture = for { - _ <- writerFuture - readerResults <- Future.sequence(readerFutures) - } yield readerResults - - val results = combinedFuture.futureValue - - // Verify the results - results.foreach { items => - items should contain(initialContent) - (1 to numberOfWrites).foreach { i => - items should contain(s"Read-Write Test Write $i") - } - } - } - - it should "handle writing after reopening the file" in { - fileDocument.open() - fileDocument.putOne("First Write") - fileDocument.close() - - // Reopen and write again - fileDocument.open() - fileDocument.putOne("Second Write") - fileDocument.close() - - val items = fileDocument.get().toList - - items should contain(initialContent) - items should contain("First Write") - items should contain("Second Write") - } -} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala index 4a3e6bee47e..ce28a59018e 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala @@ -9,16 +9,16 @@ import org.scalatest.matchers.should.Matchers class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - var partitionDocument: PartitionedFileDocument[ItemizedFileDocument[String], String] = _ + var partitionDocument: PartitionedFileDocument[ArrowFileDocument[String], String] = _ val numOfPartitions = 3 val partitionId: String = "partition_doc_test" before { // Initialize the PartitionDocument with a base ID and number of partitions - partitionDocument = new PartitionedFileDocument[ItemizedFileDocument[String], String]( + partitionDocument = new PartitionedFileDocument[ArrowFileDocument[String], String]( partitionId, numOfPartitions, - uri => new ItemizedFileDocument[String](uri) + uri => new ArrowFileDocument[String](uri) ) } @@ -30,7 +30,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA "PartitionDocument" should "create and write to each partition directly" in { for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ItemizedFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Data for partition $i") fileDoc.close() @@ -46,7 +46,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ItemizedFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Content in partition $i") fileDoc.close() @@ -63,7 +63,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ItemizedFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Some data in partition $i") fileDoc.close() @@ -86,7 +86,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA val futures = (0 until numOfPartitions).map { i => Future { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ItemizedFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI) fileDoc.open() fileDoc.putOne(s"Concurrent write to partition $i") fileDoc.close() From e2c551568e819e5c7e2eb9fe8db7b90d537174ef Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 12 Dec 2024 15:24:03 -0800 Subject: [PATCH 10/49] pass the compilation --- .../result/ArrowFileDocumentSpec.scala | 11 +++++++ .../result/PartitionedFileDocumentSpec.scala | 31 ++++++++++++++++--- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala index e8185e1a380..23693063922 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala @@ -14,6 +14,17 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future import scala.jdk.CollectionConverters._ +object ArrowFileDocumentSpec { + def stringSerializer(item: String, index: Int, root: VectorSchemaRoot): Unit = { + val vector = root.getVector("data").asInstanceOf[VarCharVector] + vector.setSafe(index, item.getBytes("UTF-8")) + } + + def stringDeserializer(index: Int, root: VectorSchemaRoot): String = { + new String(root.getVector("data").asInstanceOf[VarCharVector].get(index)) + } +} + class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { val stringArrowSchema = new Schema(List( diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala index ce28a59018e..4a3840fbe76 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala @@ -1,14 +1,35 @@ package edu.uci.ics.amber.core.storage.result +import edu.uci.ics.amber.core.storage.result.ArrowFileDocumentSpec.{stringDeserializer, stringSerializer} import edu.uci.ics.amber.core.storage.result.PartitionedFileDocument.getPartitionURI +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema} +import org.apache.arrow.vector.{VarCharVector, VectorSchemaRoot} import org.apache.commons.vfs2.{FileObject, VFS} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures.convertScalaFuture import org.scalatest.matchers.should.Matchers +import scala.jdk.CollectionConverters.IterableHasAsJava + +object ArrowFileDocumentSpec { + def stringSerializer(item: String, index: Int, root: VectorSchemaRoot): Unit = { + val vector = root.getVector("data").asInstanceOf[VarCharVector] + vector.setSafe(index, item.getBytes("UTF-8")) + } + + def stringDeserializer(index: Int, root: VectorSchemaRoot): String = { + new String(root.getVector("data").asInstanceOf[VarCharVector].get(index)) + } +} + + class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { + val stringArrowSchema = new Schema(List( + Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) + ).asJava) + var partitionDocument: PartitionedFileDocument[ArrowFileDocument[String], String] = _ val numOfPartitions = 3 val partitionId: String = "partition_doc_test" @@ -18,7 +39,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA partitionDocument = new PartitionedFileDocument[ArrowFileDocument[String], String]( partitionId, numOfPartitions, - uri => new ArrowFileDocument[String](uri) + uri => new ArrowFileDocument[String](uri, stringArrowSchema, stringSerializer, stringDeserializer) ) } @@ -30,7 +51,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA "PartitionDocument" should "create and write to each partition directly" in { for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) fileDoc.open() fileDoc.putOne(s"Data for partition $i") fileDoc.close() @@ -46,7 +67,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) fileDoc.open() fileDoc.putOne(s"Content in partition $i") fileDoc.close() @@ -63,7 +84,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) fileDoc.open() fileDoc.putOne(s"Some data in partition $i") fileDoc.close() @@ -86,7 +107,7 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA val futures = (0 until numOfPartitions).map { i => Future { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI) + val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) fileDoc.open() fileDoc.putOne(s"Concurrent write to partition $i") fileDoc.close() From c17a54e068b5161e45298e46e217b9ae317eba2e Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 13 Dec 2024 16:13:49 -0800 Subject: [PATCH 11/49] finish arrow document --- .../storage/result/ArrowFileDocument.scala | 188 ++++++++---------- .../result/ArrowFileDocumentSpec.scala | 96 +++++---- .../result/PartitionedFileDocumentSpec.scala | 45 ++++- 3 files changed, 177 insertions(+), 152 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala index dc33304bbc9..afbe38b76cc 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/ArrowFileDocument.scala @@ -16,18 +16,22 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Using class ArrowFileDocument[T]( - val uri: URI, - val arrowSchema: Schema, - val serializer: (T, Int, VectorSchemaRoot) => Unit, - val deserializer: (Int, VectorSchemaRoot) => T - ) extends VirtualDocument[T] with BufferedItemWriter[T] { + val uri: URI, + val arrowSchema: Schema, + val serializer: (T, Int, VectorSchemaRoot) => Unit, + val deserializer: (Int, VectorSchemaRoot) => T +) extends VirtualDocument[T] + with BufferedItemWriter[T] { private val file: FileObject = VFS.getManager.resolveFile(uri) private val lock = new ReentrantReadWriteLock() - private val allocator = new RootAllocator() private val buffer = new ArrayBuffer[T]() override val bufferSize: Int = 1024 + private var arrowRootallocator: RootAllocator = _ + private var arrowVectorSchemaRoot: VectorSchemaRoot = _ + private var arrowFileWriter: ArrowFileWriter = _ + // Initialize the file if it doesn't exist withWriteLock { if (!file.exists()) { @@ -39,127 +43,105 @@ class ArrowFileDocument[T]( } } - // Utility function to wrap code block with read lock private def withReadLock[M](block: => M): M = { lock.readLock().lock() try block finally lock.readLock().unlock() } - // Utility function to wrap code block with write lock private def withWriteLock[M](block: => M): M = { lock.writeLock().lock() try block finally lock.writeLock().unlock() } - override def putOne(item: T): Unit = withWriteLock { - buffer.append(item) - if (buffer.size >= bufferSize) { - flushBuffer() + override def open(): Unit = + withWriteLock { + buffer.clear() + arrowRootallocator = new RootAllocator() + arrowVectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, arrowRootallocator) + val outputStream = new FileOutputStream(file.getURL.getPath) + arrowFileWriter = new ArrowFileWriter(arrowVectorSchemaRoot, null, outputStream.getChannel) + arrowFileWriter.start() } - } - - override def removeOne(item: T): Unit = withWriteLock { - buffer -= item - } - - /** Write buffered items to the file and clear the buffer */ - private def flushBuffer(): Unit = withWriteLock { - val outputStream = new FileOutputStream(file.getURL.getPath, true) - Using.Manager { use => - val root = VectorSchemaRoot.create(arrowSchema, allocator) - val writer = new ArrowFileWriter(root, null, outputStream.getChannel) - use(writer) - use(root) - writer.start() - - buffer.zipWithIndex.foreach { case (item, index) => - serializer(item, index, root) + override def putOne(item: T): Unit = + withWriteLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() } - - root.setRowCount(buffer.size) - writer.writeBatch() - buffer.clear() - writer.end() } - } - - /** Open the writer (clear the buffer) */ - override def open(): Unit = withWriteLock { - buffer.clear() - } - /** Close the writer, flushing any remaining buffered items */ - override def close(): Unit = withWriteLock { - if (buffer.nonEmpty) { - flushBuffer() + override def removeOne(item: T): Unit = + withWriteLock { + buffer -= item } - allocator.close() - } - /** Get an iterator of data items of type T */ - private def getIterator: Iterator[T] = withReadLock { - val path = Paths.get(file.getURL.toURI) - val channel: SeekableByteChannel = FileChannel.open(path, StandardOpenOption.READ) - val reader = new ArrowFileReader(channel, allocator) - val root = reader.getVectorSchemaRoot - - new Iterator[T] { - private var currentIndex = 0 - private var currentBatchLoaded = reader.loadNextBatch() - - private def loadNextBatch(): Boolean = { - currentBatchLoaded = reader.loadNextBatch() - currentIndex = 0 - currentBatchLoaded + private def flushBuffer(): Unit = + withWriteLock { + if (buffer.nonEmpty) { + buffer.zipWithIndex.foreach { + case (item, index) => + serializer(item, index, arrowVectorSchemaRoot) + } + arrowVectorSchemaRoot.setRowCount(buffer.size) + arrowFileWriter.writeBatch() + buffer.clear() + arrowVectorSchemaRoot.clear() } + } - override def hasNext: Boolean = currentIndex < root.getRowCount || loadNextBatch() - - override def next(): T = { - if (!hasNext) throw new NoSuchElementException("No more elements") - val item = deserializer(currentIndex, root) - currentIndex += 1 - item + override def close(): Unit = + withWriteLock { + if (buffer.nonEmpty) { + flushBuffer() + } + if (arrowFileWriter != null) { + arrowFileWriter.end() + arrowFileWriter.close() } + if (arrowVectorSchemaRoot != null) arrowVectorSchemaRoot.close() + if (arrowRootallocator != null) arrowRootallocator.close() } - } - - /** Get the ith data item */ - override def getItem(i: Int): T = withReadLock { - getIterator.drop(i).next() - } - - /** Get a range of data items */ - override def getRange(from: Int, until: Int): Iterator[T] = withReadLock { - getIterator.slice(from, until) - } - - /** Get items after a certain offset */ - override def getAfter(offset: Int): Iterator[T] = withReadLock { - getIterator.drop(offset + 1) - } - - /** Get the total count of items */ - override def getCount: Long = withReadLock { - getIterator.size - } - - /** Get all items as an iterator */ - override def get(): Iterator[T] = withReadLock { - getIterator - } - /** Physically remove the file */ - override def clear(): Unit = withWriteLock { - if (file.exists()) { - file.delete() - } else { - throw new RuntimeException(s"File $uri doesn't exist") + override def get(): Iterator[T] = + withReadLock { + val path = Paths.get(file.getURL.toURI) + val allocator = new RootAllocator() + val channel: SeekableByteChannel = FileChannel.open(path, StandardOpenOption.READ) + val reader = new ArrowFileReader(channel, allocator) + val root = reader.getVectorSchemaRoot + + new Iterator[T] { + private var currentIndex = 0 + private var currentBatchLoaded = reader.loadNextBatch() + + private def loadNextBatch(): Boolean = { + currentBatchLoaded = reader.loadNextBatch() + currentIndex = 0 + currentBatchLoaded + } + + override def hasNext: Boolean = currentIndex < root.getRowCount || loadNextBatch() + + override def next(): T = { + if (!hasNext) throw new NoSuchElementException("No more elements") + val item = deserializer(currentIndex, root) + currentIndex += 1 + item + } + } } - } override def getURI: URI = uri -} \ No newline at end of file + + override def clear(): Unit = + withWriteLock { + if (file.exists()) { + file.delete() + } else { + throw new RuntimeException(s"File $uri doesn't exist") + } + } +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala index 23693063922..1b0794025a3 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/ArrowFileDocumentSpec.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.storage.result import edu.uci.ics.amber.core.storage.result.ArrowFileDocument -import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} -import org.apache.arrow.vector.{IntVector, VarCharVector, VectorSchemaRoot} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema} +import org.apache.arrow.vector.{VarCharVector, VectorSchemaRoot} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures.convertScalaFuture import org.scalatest.flatspec.AnyFlatSpec @@ -27,22 +27,29 @@ object ArrowFileDocumentSpec { class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - val stringArrowSchema = new Schema(List( - Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) - ).asJava) - - def stringSerializer(item: String, index: Int, root: VectorSchemaRoot): Unit = { - val vector = root.getVector("data").asInstanceOf[VarCharVector] - vector.setSafe(index, item.getBytes("UTF-8")) - } - - def stringDeserializer(index: Int, root: VectorSchemaRoot): String = { - new String(root.getVector("data").asInstanceOf[VarCharVector].get(index)) - } + val stringArrowSchema = new Schema( + List( + Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) + ).asJava + ) def createDocument(): ArrowFileDocument[String] = { val tempPath = Files.createTempFile("arrow_test", ".arrow") - new ArrowFileDocument[String](tempPath.toUri, stringArrowSchema, stringSerializer, stringDeserializer) + new ArrowFileDocument[String]( + tempPath.toUri, + stringArrowSchema, + ArrowFileDocumentSpec.stringSerializer, + ArrowFileDocumentSpec.stringDeserializer + ) + } + + def openDocument(uri: URI): ArrowFileDocument[String] = { + new ArrowFileDocument[String]( + uri, + stringArrowSchema, + ArrowFileDocumentSpec.stringSerializer, + ArrowFileDocumentSpec.stringDeserializer + ) } def deleteDocument(doc: ArrowFileDocument[String]): Unit = { @@ -57,8 +64,7 @@ class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte doc.close() val items = doc.get().toList - items should contain("Buffered Item 1") - items should contain("Buffered Item 2") + items should contain theSameElementsAs List("Buffered Item 1", "Buffered Item 2") deleteDocument(doc) } @@ -72,24 +78,26 @@ class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte doc.close() val items = doc.get().toList - largeBuffer.foreach { item => - items should contain(item) - } + items should contain theSameElementsAs largeBuffer deleteDocument(doc) } - it should "allow removing items from the buffer" in { + it should "override file content when reopened for writing" in { val doc = createDocument() + + // First write doc.open() - doc.putOne("Item to keep") - doc.putOne("Item to remove") - doc.removeOne("Item to remove") + doc.putOne("First Write") + doc.close() + + // Second write should override the first one + doc.open() + doc.putOne("Second Write") doc.close() val items = doc.get().toList - items should contain("Item to keep") - items should not contain "Item to remove" + items should contain only "Second Write" deleteDocument(doc) } @@ -98,15 +106,15 @@ class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte val doc = createDocument() val numberOfThreads = 5 + doc.open() val futures = (1 to numberOfThreads).map { i => Future { - doc.open() doc.putOne(s"Content from thread $i") - doc.close() } } Future.sequence(futures).futureValue + doc.close() val items = doc.get().toList (1 to numberOfThreads).foreach { i => @@ -116,23 +124,33 @@ class ArrowFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfte deleteDocument(doc) } - it should "handle writing after reopening the file" in { + it should "handle concurrent reads and writes safely" in { val doc = createDocument() - // First write + // Writer thread to add items doc.open() - doc.putOne("First Write") - doc.close() + val writerFuture = Future { + (1 to 10).foreach { i => + doc.putOne(s"Write $i") + } + } - // Second write - doc.open() - doc.putOne("Second Write") + // Reader threads to read items concurrently + val readerFutures = (1 to 3).map { _ => + Future { + doc.get().toList + } + } + + Future.sequence(readerFutures).futureValue + writerFuture.futureValue doc.close() - val items = doc.get().toList - items should contain("First Write") - items should contain("Second Write") + val finalItems = doc.get().toList + (1 to 10).foreach { i => + finalItems should contain(s"Write $i") + } deleteDocument(doc) } -} \ No newline at end of file +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala index 4a3840fbe76..1260eb3f4e0 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/PartitionedFileDocumentSpec.scala @@ -1,6 +1,9 @@ package edu.uci.ics.amber.core.storage.result -import edu.uci.ics.amber.core.storage.result.ArrowFileDocumentSpec.{stringDeserializer, stringSerializer} +import edu.uci.ics.amber.core.storage.result.ArrowFileDocumentSpec.{ + stringDeserializer, + stringSerializer +} import edu.uci.ics.amber.core.storage.result.PartitionedFileDocument.getPartitionURI import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema} import org.apache.arrow.vector.{VarCharVector, VectorSchemaRoot} @@ -23,12 +26,13 @@ object ArrowFileDocumentSpec { } } - class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeAndAfter { - val stringArrowSchema = new Schema(List( - Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) - ).asJava) + val stringArrowSchema = new Schema( + List( + Field.nullablePrimitive("data", ArrowType.Utf8.INSTANCE) + ).asJava + ) var partitionDocument: PartitionedFileDocument[ArrowFileDocument[String], String] = _ val numOfPartitions = 3 @@ -39,7 +43,8 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA partitionDocument = new PartitionedFileDocument[ArrowFileDocument[String], String]( partitionId, numOfPartitions, - uri => new ArrowFileDocument[String](uri, stringArrowSchema, stringSerializer, stringDeserializer) + uri => + new ArrowFileDocument[String](uri, stringArrowSchema, stringSerializer, stringDeserializer) ) } @@ -51,7 +56,12 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA "PartitionDocument" should "create and write to each partition directly" in { for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) + val fileDoc = new ArrowFileDocument[String]( + partitionURI, + stringArrowSchema, + stringSerializer, + stringDeserializer + ) fileDoc.open() fileDoc.putOne(s"Data for partition $i") fileDoc.close() @@ -67,7 +77,12 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) + val fileDoc = new ArrowFileDocument[String]( + partitionURI, + stringArrowSchema, + stringSerializer, + stringDeserializer + ) fileDoc.open() fileDoc.putOne(s"Content in partition $i") fileDoc.close() @@ -84,7 +99,12 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA // Write some data directly to each partition for (i <- 0 until numOfPartitions) { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) + val fileDoc = new ArrowFileDocument[String]( + partitionURI, + stringArrowSchema, + stringSerializer, + stringDeserializer + ) fileDoc.open() fileDoc.putOne(s"Some data in partition $i") fileDoc.close() @@ -107,7 +127,12 @@ class PartitionedFileDocumentSpec extends AnyFlatSpec with Matchers with BeforeA val futures = (0 until numOfPartitions).map { i => Future { val partitionURI = getPartitionURI(partitionId, i) - val fileDoc = new ArrowFileDocument[String](partitionURI, stringArrowSchema, stringSerializer, stringDeserializer) + val fileDoc = new ArrowFileDocument[String]( + partitionURI, + stringArrowSchema, + stringSerializer, + stringDeserializer + ) fileDoc.open() fileDoc.putOne(s"Concurrent write to partition $i") fileDoc.close() From bc38cc4c7c40a4c745f58c4f15f916302950d9cf Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 15 Dec 2024 22:28:55 -0800 Subject: [PATCH 12/49] start to add some iceberg related --- core/workflow-core/build.sbt | 6 + .../storage/result/IcebergFileDocument.scala | 105 ++++++++++++++++++ 2 files changed, 111 insertions(+) create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index 78dedbc56da..329c0e22a4d 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -111,6 +111,12 @@ val arrowDependencies = Seq( libraryDependencies ++= arrowDependencies +libraryDependencies ++= Seq( + "org.apache.iceberg" % "iceberg-api" % "1.7.1", + "org.apache.iceberg" % "iceberg-core" % "1.7.1", + "org.apache.iceberg" % "iceberg-parquet" % "1.7.1", +) + ///////////////////////////////////////////////////////////////////////////// // Additional Dependencies ///////////////////////////////////////////////////////////////////////////// diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala new file mode 100644 index 00000000000..0bda9729543 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala @@ -0,0 +1,105 @@ +package edu.uci.ics.amber.core.storage.result + +import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} +import org.apache.iceberg.{CatalogUtil, DataFile, Table} +import org.apache.iceberg.catalog.{Namespace, TableIdentifier} +import org.apache.iceberg.io.{DataTask, OutputFileFactory} +import org.apache.iceberg.parquet.Parquet + +import java.net.URI +import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.collection.convert.ImplicitConversions.`iterator asScala` +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters.IteratorHasAsScala +import scala.util.Using + +class IcebergFileDocument[T]( + val tableName: String, + val tableLocation: String, + val serializer: T => Array[Byte], + val deserializer: Array[Byte] => T + ) extends VirtualDocument[T] + with BufferedItemWriter[T] { + + private val lock = new ReentrantReadWriteLock() + private val buffer = new ArrayBuffer[T]() + override val bufferSize: Int = 1024 + + // Iceberg table reference + private val table: Table = loadOrCreateTable() + + // Utility function to wrap code block with read lock + private def withReadLock[M](block: => M): M = { + lock.readLock().lock() + try block + finally lock.readLock().unlock() + } + + // Utility function to wrap code block with write lock + private def withWriteLock[M](block: => M): M = { + lock.writeLock().lock() + try block + finally lock.writeLock().unlock() + } + + private def loadOrCreateTable(): Table = { + val catalog = CatalogUtil.loadCatalog("rest", "iceberg", Map("uri" -> tableLocation)) + val identifier = TableIdentifier.of(Namespace.of("default"), tableName) + if (!catalog.tableExists(identifier)) { + catalog.createTable(identifier, org.apache.iceberg.Schema.empty()) + } else { + catalog.loadTable(identifier) + } + } + + override def getURI: URI = new URI(tableLocation) + + override def open(): Unit = withWriteLock { + buffer.clear() + } + + override def putOne(item: T): Unit = withWriteLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() + } + } + + override def removeOne(item: T): Unit = withWriteLock { + buffer -= item + } + + private def flushBuffer(): Unit = withWriteLock { + if (buffer.nonEmpty) { + val outputFileFactory = OutputFileFactory.builderFor(table, 0, 0).build() + val outputFile = outputFileFactory.newOutputFile() + Using(Parquet.writeData(outputFile).createWriterFunc(serializer).build()) { writer => + buffer.foreach(writer.write) + } + table.refresh() // Refresh table to commit new data + buffer.clear() + } + } + + override def close(): Unit = withWriteLock { + if (buffer.nonEmpty) { + flushBuffer() + } + } + + override def clear(): Unit = withWriteLock { + table.newDelete().deleteFromRowFilter(null).commit() + } + + override def get(): Iterator[T] = withReadLock { + table.currentSnapshot() match { + case null => Iterator.empty + case snapshot => + table.newScan().planFiles().iterator().flatMap { task => + val file = task.file() + val reader = Parquet.read(file.path().toString).createReaderFunc(deserializer).build() + reader.iterator().asScala + } + } + } +} \ No newline at end of file From 51dd7cf2becaeda82ddbdf08e8f7e64989635788 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Tue, 17 Dec 2024 15:41:09 -0800 Subject: [PATCH 13/49] finish initial iceberg writer --- .../amber/core/storage/StorageConfig.scala | 4 + .../storage/result/IcebergFileDocument.scala | 105 ---------------- .../storage/result/IcebergTableWriter.scala | 115 ++++++++++++++++++ .../edu/uci/ics/amber/util/PathUtils.scala | 3 - 4 files changed, 119 insertions(+), 108 deletions(-) delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index 08e25051995..0c166584f52 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -1,5 +1,6 @@ package edu.uci.ics.amber.core.storage +import edu.uci.ics.amber.util.PathUtils.corePath import org.yaml.snakeyaml.Yaml import java.util.{Map => JMap} @@ -50,4 +51,7 @@ object StorageConfig { .asInstanceOf[Map[String, Any]]("jdbc") .asInstanceOf[Map[String, Any]]("password") .asInstanceOf[String] + + // For file storage specifics + val fileStorageDirectoryUri = corePath.resolve("amber").resolve("user-resources").resolve("workflow-results").toUri } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala deleted file mode 100644 index 0bda9729543..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergFileDocument.scala +++ /dev/null @@ -1,105 +0,0 @@ -package edu.uci.ics.amber.core.storage.result - -import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} -import org.apache.iceberg.{CatalogUtil, DataFile, Table} -import org.apache.iceberg.catalog.{Namespace, TableIdentifier} -import org.apache.iceberg.io.{DataTask, OutputFileFactory} -import org.apache.iceberg.parquet.Parquet - -import java.net.URI -import java.util.concurrent.locks.ReentrantReadWriteLock -import scala.collection.convert.ImplicitConversions.`iterator asScala` -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters.IteratorHasAsScala -import scala.util.Using - -class IcebergFileDocument[T]( - val tableName: String, - val tableLocation: String, - val serializer: T => Array[Byte], - val deserializer: Array[Byte] => T - ) extends VirtualDocument[T] - with BufferedItemWriter[T] { - - private val lock = new ReentrantReadWriteLock() - private val buffer = new ArrayBuffer[T]() - override val bufferSize: Int = 1024 - - // Iceberg table reference - private val table: Table = loadOrCreateTable() - - // Utility function to wrap code block with read lock - private def withReadLock[M](block: => M): M = { - lock.readLock().lock() - try block - finally lock.readLock().unlock() - } - - // Utility function to wrap code block with write lock - private def withWriteLock[M](block: => M): M = { - lock.writeLock().lock() - try block - finally lock.writeLock().unlock() - } - - private def loadOrCreateTable(): Table = { - val catalog = CatalogUtil.loadCatalog("rest", "iceberg", Map("uri" -> tableLocation)) - val identifier = TableIdentifier.of(Namespace.of("default"), tableName) - if (!catalog.tableExists(identifier)) { - catalog.createTable(identifier, org.apache.iceberg.Schema.empty()) - } else { - catalog.loadTable(identifier) - } - } - - override def getURI: URI = new URI(tableLocation) - - override def open(): Unit = withWriteLock { - buffer.clear() - } - - override def putOne(item: T): Unit = withWriteLock { - buffer.append(item) - if (buffer.size >= bufferSize) { - flushBuffer() - } - } - - override def removeOne(item: T): Unit = withWriteLock { - buffer -= item - } - - private def flushBuffer(): Unit = withWriteLock { - if (buffer.nonEmpty) { - val outputFileFactory = OutputFileFactory.builderFor(table, 0, 0).build() - val outputFile = outputFileFactory.newOutputFile() - Using(Parquet.writeData(outputFile).createWriterFunc(serializer).build()) { writer => - buffer.foreach(writer.write) - } - table.refresh() // Refresh table to commit new data - buffer.clear() - } - } - - override def close(): Unit = withWriteLock { - if (buffer.nonEmpty) { - flushBuffer() - } - } - - override def clear(): Unit = withWriteLock { - table.newDelete().deleteFromRowFilter(null).commit() - } - - override def get(): Iterator[T] = withReadLock { - table.currentSnapshot() match { - case null => Iterator.empty - case snapshot => - table.newScan().planFiles().iterator().flatMap { task => - val file = task.file() - val reader = Parquet.read(file.path().toString).createReaderFunc(deserializer).build() - reader.iterator().asScala - } - } - } -} \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala new file mode 100644 index 00000000000..80ab386a9df --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala @@ -0,0 +1,115 @@ +package edu.uci.ics.amber.core.storage.result + +import edu.uci.ics.amber.core.storage.StorageConfig +import edu.uci.ics.amber.core.storage.model.BufferedItemWriter +import org.apache.iceberg.{CatalogUtil, DataFile, Table} +import org.apache.iceberg.catalog.{Namespace, TableIdentifier} +import org.apache.iceberg.io.{DataWriter, OutputFile} +import org.apache.iceberg.parquet.Parquet +import org.apache.iceberg.data.GenericRecord + +import java.util.UUID +import java.util.concurrent.locks.ReentrantLock +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +class IcebergTableWriter[T]( + val tableName: String, + val tableSchema: org.apache.iceberg.Schema, + val convertToGenericRecord: T => GenericRecord + ) extends BufferedItemWriter[T] { + + private val lock = new ReentrantLock() + private val buffer = new ArrayBuffer[T]() + override val bufferSize: Int = 1024 + + // Load the Iceberg table + private val table: Table = loadOrCreateTable() + + private def loadOrCreateTable(): Table = { + val catalog = CatalogUtil.loadCatalog( + classOf[org.apache.iceberg.jdbc.JdbcCatalog].getName, + "iceberg", + Map( + "uri" -> StorageConfig.jdbcUrl, + "warehouse" -> StorageConfig.fileStorageDirectoryUri.toString, + "jdbc.user" -> StorageConfig.jdbcUsername, + "jdbc.password" -> StorageConfig.jdbcPassword + ).asJava, + null + ) + + val identifier = TableIdentifier.of(Namespace.of("default"), tableName) + if (!catalog.tableExists(identifier)) { + catalog.createTable(identifier, tableSchema) + } else { + catalog.loadTable(identifier) + } + } + + override def open(): Unit = withLock { + buffer.clear() + } + + override def putOne(item: T): Unit = withLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() + } + } + + override def removeOne(item: T): Unit = withLock { + buffer -= item + } + + private def flushBuffer(): Unit = withLock { + if (buffer.nonEmpty) { + try { + // Create a unique file path using UUID + val filepath = s"${table.location()}/${UUID.randomUUID().toString}" + val outputFile: OutputFile = table.io().newOutputFile(filepath) + + // Create a Parquet data writer + val dataWriter: DataWriter[GenericRecord] = Parquet.writeData(outputFile) + .schema(table.schema()) + .createWriterFunc(org.apache.iceberg.data.parquet.GenericParquetWriter.buildWriter) + .overwrite() + .build() + + try { + buffer.foreach { item => + val record = convertToGenericRecord(item) + dataWriter.write(record) + } + } finally { + dataWriter.close() + } + + // Commit the new file to the table + val dataFile: DataFile = dataWriter.toDataFile + table.newAppend().appendFile(dataFile).commit() + + println(s"Flushed ${buffer.size} records to ${filepath}") + + buffer.clear() + } catch { + case e: Exception => + println(s"Error during flush: ${e.getMessage}") + e.printStackTrace() + } + } + } + + override def close(): Unit = withLock { + if (buffer.nonEmpty) { + flushBuffer() + } + } + + // Utility function to wrap code block with write lock + private def withLock[M](block: => M): M = { + lock.lock() + try block + finally lock.unlock() + } +} \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala index 1a18e59aad9..7fe89d0e8ce 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/PathUtils.scala @@ -39,9 +39,6 @@ object PathUtils { lazy val workflowCompilingServicePath: Path = corePath.resolve("workflow-compiling-service") - lazy val workflowResultsRootPath: Path = - corePath.resolve("amber").resolve("user-resources").resolve("workflow-results") - private lazy val datasetsRootPath = corePath.resolve("amber").resolve("user-resources").resolve("datasets") From 481c437aebc846f3911a292852fef822b3312c71 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Wed, 18 Dec 2024 22:16:54 -0800 Subject: [PATCH 14/49] finish initial version of iceberg --- core/build.sbt | 8 +- core/workflow-core/build.sbt | 2 + .../amber/core/storage/StorageConfig.scala | 3 +- .../storage/result/IcebergTableWriter.scala | 115 ----------- .../result/PartitionedFileDocument.scala | 6 +- .../result/iceberg/IcebergDocument.scala | 110 +++++++++++ .../result/iceberg/IcebergTableWriter.scala | 103 ++++++++++ .../result/iceberg/fileio/LocalFileIO.scala | 46 +++++ .../iceberg/fileio/LocalInputFile.scala | 54 ++++++ .../iceberg/fileio/LocalOutputFile.scala | 57 ++++++ .../edu/uci/ics/amber/util/IcebergUtil.scala | 178 ++++++++++++++++++ .../result/iceberg/IcebergDocumentSpec.scala | 139 ++++++++++++++ 12 files changed, 701 insertions(+), 120 deletions(-) delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala diff --git a/core/build.sbt b/core/build.sbt index 648a9a5d7f3..bd24d6e42bf 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -3,7 +3,13 @@ lazy val WorkflowCore = (project in file("workflow-core")) .dependsOn(DAO) .configs(Test) .dependsOn(DAO % "test->test") // test scope dependency -lazy val WorkflowOperator = (project in file("workflow-operator")).dependsOn(WorkflowCore) +lazy val WorkflowOperator = (project in file("workflow-operator")) + .dependsOn(WorkflowCore) + .settings( + dependencyOverrides ++= Seq( + "org.apache.commons" % "commons-compress" % "1.23.0", // because of the dependency introduced by iceberg + ) + ) lazy val WorkflowCompilingService = (project in file("workflow-compiling-service")) .dependsOn(WorkflowOperator) .settings( diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index 329c0e22a4d..eb0ebad7e11 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -115,6 +115,8 @@ libraryDependencies ++= Seq( "org.apache.iceberg" % "iceberg-api" % "1.7.1", "org.apache.iceberg" % "iceberg-core" % "1.7.1", "org.apache.iceberg" % "iceberg-parquet" % "1.7.1", + "org.apache.iceberg" % "iceberg-data" % "1.7.1", + "org.apache.hadoop" % "hadoop-client" % "3.3.1", ) ///////////////////////////////////////////////////////////////////////////// diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index 0c166584f52..92a676032e3 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -53,5 +53,6 @@ object StorageConfig { .asInstanceOf[String] // For file storage specifics - val fileStorageDirectoryUri = corePath.resolve("amber").resolve("user-resources").resolve("workflow-results").toUri + val fileStorageDirectoryUri = + corePath.resolve("amber").resolve("user-resources").resolve("workflow-results").toUri } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala deleted file mode 100644 index 80ab386a9df..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/IcebergTableWriter.scala +++ /dev/null @@ -1,115 +0,0 @@ -package edu.uci.ics.amber.core.storage.result - -import edu.uci.ics.amber.core.storage.StorageConfig -import edu.uci.ics.amber.core.storage.model.BufferedItemWriter -import org.apache.iceberg.{CatalogUtil, DataFile, Table} -import org.apache.iceberg.catalog.{Namespace, TableIdentifier} -import org.apache.iceberg.io.{DataWriter, OutputFile} -import org.apache.iceberg.parquet.Parquet -import org.apache.iceberg.data.GenericRecord - -import java.util.UUID -import java.util.concurrent.locks.ReentrantLock -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ - -class IcebergTableWriter[T]( - val tableName: String, - val tableSchema: org.apache.iceberg.Schema, - val convertToGenericRecord: T => GenericRecord - ) extends BufferedItemWriter[T] { - - private val lock = new ReentrantLock() - private val buffer = new ArrayBuffer[T]() - override val bufferSize: Int = 1024 - - // Load the Iceberg table - private val table: Table = loadOrCreateTable() - - private def loadOrCreateTable(): Table = { - val catalog = CatalogUtil.loadCatalog( - classOf[org.apache.iceberg.jdbc.JdbcCatalog].getName, - "iceberg", - Map( - "uri" -> StorageConfig.jdbcUrl, - "warehouse" -> StorageConfig.fileStorageDirectoryUri.toString, - "jdbc.user" -> StorageConfig.jdbcUsername, - "jdbc.password" -> StorageConfig.jdbcPassword - ).asJava, - null - ) - - val identifier = TableIdentifier.of(Namespace.of("default"), tableName) - if (!catalog.tableExists(identifier)) { - catalog.createTable(identifier, tableSchema) - } else { - catalog.loadTable(identifier) - } - } - - override def open(): Unit = withLock { - buffer.clear() - } - - override def putOne(item: T): Unit = withLock { - buffer.append(item) - if (buffer.size >= bufferSize) { - flushBuffer() - } - } - - override def removeOne(item: T): Unit = withLock { - buffer -= item - } - - private def flushBuffer(): Unit = withLock { - if (buffer.nonEmpty) { - try { - // Create a unique file path using UUID - val filepath = s"${table.location()}/${UUID.randomUUID().toString}" - val outputFile: OutputFile = table.io().newOutputFile(filepath) - - // Create a Parquet data writer - val dataWriter: DataWriter[GenericRecord] = Parquet.writeData(outputFile) - .schema(table.schema()) - .createWriterFunc(org.apache.iceberg.data.parquet.GenericParquetWriter.buildWriter) - .overwrite() - .build() - - try { - buffer.foreach { item => - val record = convertToGenericRecord(item) - dataWriter.write(record) - } - } finally { - dataWriter.close() - } - - // Commit the new file to the table - val dataFile: DataFile = dataWriter.toDataFile - table.newAppend().appendFile(dataFile).commit() - - println(s"Flushed ${buffer.size} records to ${filepath}") - - buffer.clear() - } catch { - case e: Exception => - println(s"Error during flush: ${e.getMessage}") - e.printStackTrace() - } - } - } - - override def close(): Unit = withLock { - if (buffer.nonEmpty) { - flushBuffer() - } - } - - // Utility function to wrap code block with write lock - private def withLock[M](block: => M): M = { - lock.lock() - try block - finally lock.unlock() - } -} \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala index fbb67703f02..8100d9258e2 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/PartitionedFileDocument.scala @@ -1,8 +1,8 @@ package edu.uci.ics.amber.core.storage.result +import edu.uci.ics.amber.core.storage.StorageConfig import edu.uci.ics.amber.core.storage.model.VirtualDocument import edu.uci.ics.amber.core.storage.result.PartitionedFileDocument.getPartitionURI -import edu.uci.ics.amber.util.PathUtils.workflowResultsRootPath import org.apache.commons.vfs2.VFS import java.net.URI @@ -14,7 +14,7 @@ object PartitionedFileDocument { * Utility function to generate the partition URI by index. */ def getPartitionURI(id: String, i: Int): URI = { - workflowResultsRootPath.resolve(id).resolve(s"partition$i").toUri + StorageConfig.fileStorageDirectoryUri.resolve(id).resolve(s"partition$i") } } @@ -36,7 +36,7 @@ class PartitionedFileDocument[R <: VirtualDocument[T], T >: Null <: AnyRef]( // Use round-robin to decide which partition to go to when reading the i-th item private def getPartitionIndex(i: Int): Int = i % numOfPartition - override def getURI: URI = workflowResultsRootPath.resolve(id).toUri + override def getURI: URI = StorageConfig.fileStorageDirectoryUri.resolve(id) override def getItem(i: Int): T = { mutex.lock() diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala new file mode 100644 index 00000000000..96dbaa51800 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -0,0 +1,110 @@ +package edu.uci.ics.amber.core.storage.result.iceberg + +import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} +import edu.uci.ics.amber.util.IcebergUtil +import org.apache.iceberg.Table +import org.apache.iceberg.catalog.{Catalog, TableIdentifier} +import org.apache.iceberg.data.{IcebergGenerics, Record} +import org.apache.iceberg.io.CloseableIterable + +import java.net.URI +import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.jdk.CollectionConverters._ + +class IcebergDocument[T >: Null <: AnyRef]( + val catalog: Catalog, + val tableNamespace: String, + val tableName: String, + val tableSchema: org.apache.iceberg.Schema, + val serde: T => Record, + val deserde: (org.apache.iceberg.Schema, Record) => T +) extends VirtualDocument[T] { + + private val lock = new ReentrantReadWriteLock() + + /** + * Returns the URI of the table location. + */ + override def getURI: URI = { + val table = IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) + URI.create(table.location()) + } + + /** + * Deletes the table. + */ + override def clear(): Unit = + withWriteLock { + val identifier = TableIdentifier.of(tableNamespace, tableName) + if (catalog.tableExists(identifier)) { + catalog.dropTable(identifier) + } + } + + /** + * Returns an iterator that iterates over all records in the table, including new records + * from concurrent writers as they commit. + */ + override def get(): Iterator[T] = + new Iterator[T] { + private val table = + IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) + private var currentSnapshot = table.currentSnapshot() + private var recordIterator = loadRecords() + + /** + * Loads all records from the current snapshot. + */ + private def loadRecords(): Iterator[T] = { + if (currentSnapshot != null) { + try { + val records: CloseableIterable[Record] = IcebergGenerics.read(table).build() + records.iterator().asScala.map(record => deserde(tableSchema, record)) + } catch { + case _: java.io.FileNotFoundException => + println("Metadata file not found. Returning an empty iterator.") + Iterator.empty + case e: Exception => + println(s"Error during record loading: ${e.getMessage}") + e.printStackTrace() + Iterator.empty + } + } else { + Iterator.empty + } + } + + override def hasNext: Boolean = { + if (recordIterator.hasNext) { + true + } else { + // Refresh the table and check for new commits + table.refresh() + val newSnapshot = table.currentSnapshot() + if (newSnapshot != currentSnapshot) { + currentSnapshot = newSnapshot + recordIterator = loadRecords() + recordIterator.hasNext + } else { + false + } + } + } + + override def next(): T = recordIterator.next() + } + + /** + * Returns a BufferedItemWriter for writing data to the table. + */ + override def writer(): BufferedItemWriter[T] = { + new IcebergTableWriter[T](catalog, tableNamespace, tableName, tableSchema, serde) + } + + // Utility function to wrap code block with write lock + private def withWriteLock[M](block: => M): M = { + lock.writeLock().lock() + try block + finally lock.writeLock().unlock() + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala new file mode 100644 index 00000000000..5c8f27ecc06 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -0,0 +1,103 @@ +package edu.uci.ics.amber.core.storage.result.iceberg + +import edu.uci.ics.amber.core.storage.model.BufferedItemWriter +import edu.uci.ics.amber.util.IcebergUtil +import org.apache.iceberg.{Schema, Table} +import org.apache.iceberg.catalog.{Catalog, TableIdentifier} +import org.apache.iceberg.data.Record +import org.apache.iceberg.data.parquet.GenericParquetWriter +import org.apache.iceberg.io.{DataWriter, OutputFile} +import org.apache.iceberg.parquet.Parquet + +import java.util.UUID +import java.util.concurrent.locks.ReentrantLock +import scala.collection.mutable.ArrayBuffer + +class IcebergTableWriter[T]( + val catalog: Catalog, + val tableNamespace: String, + val tableName: String, + val tableSchema: Schema, + val serde: T => Record +) extends BufferedItemWriter[T] { + + private val lock = new ReentrantLock() + private val buffer = new ArrayBuffer[T]() + override val bufferSize: Int = 1024 + + // Load the Iceberg table + private val table: Table = + IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) + + override def open(): Unit = + withLock { + buffer.clear() + } + + override def putOne(item: T): Unit = + withLock { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() + } + } + + override def removeOne(item: T): Unit = + withLock { + buffer -= item + } + + private def flushBuffer(): Unit = + withLock { + if (buffer.nonEmpty) { + try { + // Create a unique file path using UUID + val filepath = s"${table.location()}/${UUID.randomUUID().toString}" + val outputFile: OutputFile = table.io().newOutputFile(filepath) + + // Create a Parquet data writer + val dataWriter: DataWriter[Record] = Parquet + .writeData(outputFile) + .forTable(table) + .createWriterFunc(GenericParquetWriter.buildWriter) + .overwrite() + .build() + + try { + buffer.foreach { item => + val record = serde(item) + dataWriter.write(record) + } + } finally { + dataWriter.close() + } + + // Commit the new file to the table + val dataFile = dataWriter.toDataFile + table.newAppend().appendFile(dataFile).commit() + + println(s"Flushed ${buffer.size} records to ${filepath}") + + buffer.clear() + } catch { + case e: Exception => + println(s"Error during flush: ${e.getMessage}") + e.printStackTrace() + } + } + } + + override def close(): Unit = + withLock { + if (buffer.nonEmpty) { + flushBuffer() + } + } + + // Utility function to wrap code block with write lock + private def withLock[M](block: => M): M = { + lock.lock() + try block + finally lock.unlock() + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala new file mode 100644 index 00000000000..19dd6af7102 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala @@ -0,0 +1,46 @@ +package edu.uci.ics.amber.core.storage.result.iceberg.fileio + +import org.apache.iceberg.io.{FileIO, InputFile, OutputFile} +import org.apache.iceberg.exceptions.RuntimeIOException + +import java.io.{File, IOException, RandomAccessFile} +import java.nio.file.{Files, Paths, StandardOpenOption} +import java.util.concurrent.atomic.AtomicBoolean + +class LocalFileIO extends FileIO { + + private val isClosed = new AtomicBoolean(false) + + override def newInputFile(path: String): InputFile = { + ensureNotClosed() + new LocalInputFile(path) + } + + override def newOutputFile(path: String): OutputFile = { + ensureNotClosed() + new LocalOutputFile(path) + } + + override def deleteFile(path: String): Unit = { + ensureNotClosed() + try { + Files.deleteIfExists(Paths.get(path)) + } catch { + case e: IOException => throw new RuntimeIOException(e, s"Failed to delete file: $path") + } + } + + override def initialize(properties: java.util.Map[String, String]): Unit = { + // No special initialization required for local file system + } + + override def close(): Unit = { + isClosed.set(true) + } + + private def ensureNotClosed(): Unit = { + if (isClosed.get()) { + throw new IllegalStateException("Cannot use LocalFileIO after it has been closed") + } + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala new file mode 100644 index 00000000000..9f1ca29df6b --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala @@ -0,0 +1,54 @@ +package edu.uci.ics.amber.core.storage.result.iceberg.fileio + +import org.apache.iceberg.io.{InputFile, SeekableInputStream} +import org.apache.iceberg.exceptions.RuntimeIOException + +import java.io.{File, FileInputStream, IOException, RandomAccessFile} +import java.nio.file.{Files, Paths} + +class LocalInputFile(path: String) extends InputFile { + + override def location(): String = path + + override def getLength(): Long = { + try { + Files.size(Paths.get(path)) + } catch { + case e: IOException => throw new RuntimeIOException(e, s"Failed to get length of file: $path") + } + } + + override def newStream(): SeekableInputStream = { + try { + new SeekableFileInputStream(path) + } catch { + case e: IOException => + throw new RuntimeIOException(e, s"Failed to open file for reading: $path") + } + } + + override def exists(): Boolean = Files.exists(Paths.get(path)) +} + +class SeekableFileInputStream(path: String) extends SeekableInputStream { + + private val file = new RandomAccessFile(path, "r") + + override def read(): Int = file.read() + + override def read(b: Array[Byte], off: Int, len: Int): Int = file.read(b, off, len) + + override def seek(pos: Long): Unit = file.seek(pos) + + override def getPos: Long = file.getFilePointer + + override def close(): Unit = file.close() + + override def skip(n: Long): Long = { + val currentPos = file.getFilePointer + val fileLength = file.length() + val newPos = Math.min(currentPos + n, fileLength) + file.seek(newPos) + newPos - currentPos + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala new file mode 100644 index 00000000000..fa959349d76 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala @@ -0,0 +1,57 @@ +package edu.uci.ics.amber.core.storage.result.iceberg.fileio + +import org.apache.iceberg.io.{OutputFile, PositionOutputStream} +import org.apache.iceberg.exceptions.RuntimeIOException + +import java.io.{File, FileOutputStream, IOException} +import java.nio.file.{Files, Paths, StandardOpenOption} + +class LocalOutputFile(path: String) extends OutputFile { + + override def location(): String = path + + override def create(): PositionOutputStream = { + try { + new LocalPositionOutputStream(path, append = false) + } catch { + case e: IOException => throw new RuntimeIOException(e, s"Failed to create file: $path") + } + } + + override def createOrOverwrite(): PositionOutputStream = { + try { + new LocalPositionOutputStream(path, append = false) + } catch { + case e: IOException => + throw new RuntimeIOException(e, s"Failed to create or overwrite file: $path") + } + } + + override def toInputFile: LocalInputFile = new LocalInputFile(path) +} + +class LocalPositionOutputStream(path: String, append: Boolean) extends PositionOutputStream { + + // Ensure the parent directories exist + private val file = new File(path) + if (!file.getParentFile.exists()) { + file.getParentFile.mkdirs() + } + + private val outputStream = new FileOutputStream(file, append) + private var position: Long = 0 + + override def write(b: Int): Unit = { + outputStream.write(b) + position += 1 + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + outputStream.write(b, off, len) + position += len + } + + override def getPos: Long = position + + override def close(): Unit = outputStream.close() +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala new file mode 100644 index 00000000000..4927c641a03 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -0,0 +1,178 @@ +package edu.uci.ics.amber.util + +import edu.uci.ics.amber.core.storage.StorageConfig +import edu.uci.ics.amber.core.storage.result.iceberg.fileio.LocalFileIO +import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} +import org.apache.iceberg.catalog.{Catalog, TableIdentifier} +import org.apache.iceberg.types.Types +import org.apache.iceberg.data.{GenericRecord, Record} +import org.apache.iceberg.jdbc.JdbcCatalog +import org.apache.iceberg.types.Type.PrimitiveType +import org.apache.iceberg.{CatalogProperties, Table, Schema => IcebergSchema} + +import java.net.URI +import java.sql.Timestamp +import java.time.LocalDateTime +import java.time.ZoneId +import scala.jdk.CollectionConverters._ + +object IcebergUtil { + + /** + * Creates and initializes a JdbcCatalog with the given parameters. + * + * @param catalogName The name of the catalog. + * @param warehouseUri The warehouse directory path. + * @param jdbcUri The JDBC URI for the catalog. + * @param jdbcUser The JDBC username. + * @param jdbcPassword The JDBC password. + * @return The initialized JdbcCatalog. + */ + def createJdbcCatalog( + catalogName: String, + warehouseUri: URI, + jdbcUri: String, + jdbcUser: String, + jdbcPassword: String + ): JdbcCatalog = { + val catalog = new JdbcCatalog() + catalog.initialize( + catalogName, + Map( + "warehouse" -> warehouseUri.toString, + "uri" -> jdbcUri, + "jdbc.user" -> jdbcUser, + "jdbc.password" -> jdbcPassword, + CatalogProperties.FILE_IO_IMPL -> classOf[LocalFileIO].getName + ).asJava + ) + catalog + } + + def loadOrCreateTable( + catalog: Catalog, + tableNamespace: String, + tableName: String, + tableSchema: IcebergSchema + ): Table = { + val identifier = TableIdentifier.of(tableNamespace, tableName) + if (!catalog.tableExists(identifier)) { + catalog.createTable(identifier, tableSchema) + } else { + catalog.loadTable(identifier) + } + } + + /** + * Converts a custom Amber `Schema` to an Iceberg `Schema`. + * + * @param amberSchema The custom Amber Schema. + * @return An Iceberg Schema. + */ + def toIcebergSchema(amberSchema: Schema): IcebergSchema = { + val icebergFields = amberSchema.getAttributes.zipWithIndex.map { + case (attribute, index) => + Types.NestedField.required(index + 1, attribute.getName, toIcebergType(attribute.getType)) + } + new IcebergSchema(icebergFields.asJava) + } + + /** + * Converts a custom Amber `AttributeType` to an Iceberg `Type`. + * + * @param attributeType The custom Amber AttributeType. + * @return The corresponding Iceberg Type. + */ + def toIcebergType(attributeType: AttributeType): PrimitiveType = { + attributeType match { + case AttributeType.STRING => Types.StringType.get() + case AttributeType.INTEGER => Types.IntegerType.get() + case AttributeType.LONG => Types.LongType.get() + case AttributeType.DOUBLE => Types.DoubleType.get() + case AttributeType.BOOLEAN => Types.BooleanType.get() + case AttributeType.TIMESTAMP => Types.TimestampType.withoutZone() + case AttributeType.BINARY => Types.BinaryType.get() + case AttributeType.ANY => + throw new IllegalArgumentException("ANY type is not supported in Iceberg") + } + } + + /** + * Converts a custom Amber `Tuple` to an Iceberg `GenericRecord`. + * + * @param tuple The custom Amber Tuple. + * @return An Iceberg GenericRecord. + */ + def toGenericRecord(tuple: Tuple): Record = { + // Convert the Amber schema to an Iceberg schema + val icebergSchema = toIcebergSchema(tuple.schema) + val record = GenericRecord.create(icebergSchema) + + tuple.schema.getAttributes.zipWithIndex.foreach { + case (attribute, index) => + val value = tuple.getField[AnyRef](index) match { + case ts: Timestamp => ts.toInstant.atZone(ZoneId.systemDefault()).toLocalDateTime + case other => other + } + record.setField(attribute.getName, value) + } + + record + } + + /** + * Converts an Iceberg `Record` to an Amber `Tuple`. + * + * @param record The Iceberg Record. + * @param amberSchema The corresponding Amber Schema. + * @return An Amber Tuple. + */ + def fromRecord(record: Record, amberSchema: Schema): Tuple = { + val fieldValues = amberSchema.getAttributes.map { attribute => + val value = record.getField(attribute.getName) match { + case ldt: LocalDateTime => Timestamp.valueOf(ldt) + case other => other + } + value + } + + Tuple(amberSchema, fieldValues.toArray) + } + + /** + * Converts an Iceberg `Schema` to a custom Amber `Schema`. + * + * @param icebergSchema The Iceberg Schema. + * @return The corresponding Amber Schema. + */ + def fromIcebergSchema(icebergSchema: IcebergSchema): Schema = { + val attributes = icebergSchema + .columns() + .asScala + .map { field => + new Attribute(field.name(), fromIcebergType(field.`type`().asPrimitiveType())) + } + .toList + + Schema(attributes) + } + + /** + * Converts an Iceberg `Type` to a custom Amber `AttributeType`. + * + * @param icebergType The Iceberg Type. + * @return The corresponding Amber AttributeType. + */ + def fromIcebergType(icebergType: PrimitiveType): AttributeType = { + icebergType match { + case _: Types.StringType => AttributeType.STRING + case _: Types.IntegerType => AttributeType.INTEGER + case _: Types.LongType => AttributeType.LONG + case _: Types.DoubleType => AttributeType.DOUBLE + case _: Types.BooleanType => AttributeType.BOOLEAN + case _: Types.TimestampType => AttributeType.TIMESTAMP + case _: Types.BinaryType => AttributeType.BINARY + case _ => throw new IllegalArgumentException(s"Unsupported Iceberg type: $icebergType") + } + } +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala new file mode 100644 index 00000000000..d159f2bd0d6 --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -0,0 +1,139 @@ +package edu.uci.ics.amber.storage.result.iceberg + +import edu.uci.ics.amber.core.storage.StorageConfig +import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument +import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} +import edu.uci.ics.amber.util.IcebergUtil +import org.apache.iceberg.catalog.Catalog +import org.apache.iceberg.data.Record +import org.apache.iceberg.types.Types +import org.apache.iceberg.{Schema => IcebergSchema} +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.BeforeAndAfterEach + +import java.sql.Timestamp +import java.util.UUID + +class IcebergDocumentSpec extends AnyFlatSpec with BeforeAndAfterEach { + + // Define Amber Schema + val amberSchema: Schema = Schema( + List( + new Attribute("id", AttributeType.LONG), + new Attribute("name", AttributeType.STRING), + new Attribute("score", AttributeType.DOUBLE), + new Attribute("timestamp", AttributeType.TIMESTAMP) + ) + ) + + // Define Iceberg Schema + val icebergSchema: IcebergSchema = IcebergUtil.toIcebergSchema(amberSchema) + + // Serialization function: Tuple -> Record + val serde: Tuple => Record = tuple => IcebergUtil.toGenericRecord(tuple) + + // Deserialization function: Record -> Tuple + val deserde: (IcebergSchema, Record) => Tuple = (schema, record) => + IcebergUtil.fromRecord(record, amberSchema) + + // Table name (unique for each test) + var tableName: String = _ + + var tableCatalog: Catalog = IcebergUtil.createJdbcCatalog( + "iceberg document test", + StorageConfig.fileStorageDirectoryUri, + StorageConfig.jdbcUrl, + StorageConfig.jdbcUsername, + StorageConfig.jdbcPassword + ) + + val tableNamespace = "test" + + // IcebergDocument instance + var icebergDocument: IcebergDocument[Tuple] = _ + + override def beforeEach(): Unit = { + // Generate a unique table name for each test + tableName = s"test_table_${UUID.randomUUID().toString.replace("-", "")}" + + // Initialize IcebergDocument + icebergDocument = new IcebergDocument[Tuple]( + tableCatalog, + tableNamespace, + tableName, + icebergSchema, + serde, + deserde + ) + } + + "IcebergDocument" should "write and read tuples successfully" in { + val tuple1 = Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 1L) + .add("name", AttributeType.STRING, "Alice") + .add("score", AttributeType.DOUBLE, 95.5) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build() + + val tuple2 = Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 2L) + .add("name", AttributeType.STRING, "Bob") + .add("score", AttributeType.DOUBLE, 88.0) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build() + + val tuple3 = Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 3L) + .add("name", AttributeType.STRING, "John") + .add("score", AttributeType.DOUBLE, 75.5) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build() + + val tuple4 = Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 4L) + .add("name", AttributeType.STRING, "Bob") + .add("score", AttributeType.DOUBLE, 80.0) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build() + + // Get writer and write tuples + var writer = icebergDocument.writer() + writer.open() + writer.putOne(tuple1) + writer.putOne(tuple2) + writer.close() + + // Read tuples back + var retrievedTuples = icebergDocument.get().toList + + assert(retrievedTuples.contains(tuple1)) + assert(retrievedTuples.contains(tuple2)) + assert(retrievedTuples.size == 2) + + // Get writer and write tuples + writer = icebergDocument.writer() + writer.open() + writer.putOne(tuple3) + writer.putOne(tuple4) + writer.close() + + retrievedTuples = icebergDocument.get().toList + + assert(retrievedTuples.contains(tuple1)) + assert(retrievedTuples.contains(tuple2)) + assert(retrievedTuples.contains(tuple3)) + assert(retrievedTuples.contains(tuple4)) + assert(retrievedTuples.size == 4) + + icebergDocument.clear() + } + + it should "handle empty reads gracefully" in { + val retrievedTuples = icebergDocument.get().toList + assert(retrievedTuples.isEmpty) + } +} From 0274f66f162ac106235eb8fa2a7f2050745eb112 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Wed, 18 Dec 2024 23:12:05 -0800 Subject: [PATCH 15/49] refactor test parts --- .../iceberg/fileio/LocalInputFile.scala | 2 +- .../storage/model/VirtualDocumentSpec.scala | 79 ++++++++++++ .../result/iceberg/IcebergDocumentSpec.scala | 118 ++++++------------ 3 files changed, 116 insertions(+), 83 deletions(-) create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala index 9f1ca29df6b..9301e523d9c 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala @@ -10,7 +10,7 @@ class LocalInputFile(path: String) extends InputFile { override def location(): String = path - override def getLength(): Long = { + override def getLength: Long = { try { Files.size(Paths.get(path)) } catch { diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala new file mode 100644 index 00000000000..845aa164414 --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -0,0 +1,79 @@ +package edu.uci.ics.amber.core.storage.model + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.flatspec.AnyFlatSpec + +/** + * A trait for testing VirtualDocument implementations. + * Provides common read/write test cases and hooks for subclasses to customize. + * @tparam T the type of data that the VirtualDocument handles. + */ +trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { + + /** + * Constructs the VirtualDocument instance to be tested. + * Subclasses should override this to provide their specific implementation. + */ + def getDocument: VirtualDocument[T] + + /** + * Checks if the document has been cleared. + * Subclasses should override this to provide their specific check. + * @return true if the document is cleared, false otherwise. + */ + def isDocumentCleared: Boolean + + // VirtualDocument instance for each test + var document: VirtualDocument[T] = _ + + override def beforeEach(): Unit = { + document = getDocument + } + + "VirtualDocument" should "write and read items successfully" in { + val items = generateSampleItems() + + // Get writer and write items + val writer = document.writer() + writer.open() + items.foreach(writer.putOne) + writer.close() + + // Read items back + val retrievedItems = document.get().toList + + assert(retrievedItems == items, "The retrieved items should match the written items.") + } + + it should "clear the document" in { + val items = generateSampleItems() + + // Write items + val writer = document.writer() + writer.open() + items.foreach(writer.putOne) + writer.close() + + // Ensure items are written + assert(document.get().nonEmpty, "The document should contain items before clearing.") + + // Clear the document + document.clear() + + // Check if the document is cleared + assert(isDocumentCleared, "The document should be cleared after calling clear.") + assert(document.get().isEmpty, "The document should have no items after clearing.") + } + + it should "handle empty reads gracefully" in { + val retrievedItems = document.get().toList + assert(retrievedItems.isEmpty, "Reading from an empty document should return an empty list.") + } + + /** + * Generates a sample list of items for testing. + * Subclasses should override this to provide their specific sample items. + * @return a list of sample items of type T. + */ + def generateSampleItems(): List[T] +} diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index d159f2bd0d6..c84f339a235 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -1,20 +1,19 @@ package edu.uci.ics.amber.storage.result.iceberg import edu.uci.ics.amber.core.storage.StorageConfig +import edu.uci.ics.amber.core.storage.model.VirtualDocumentSpec import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.catalog.Catalog import org.apache.iceberg.data.Record -import org.apache.iceberg.types.Types import org.apache.iceberg.{Schema => IcebergSchema} -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.BeforeAndAfterEach +import org.apache.iceberg.catalog.TableIdentifier import java.sql.Timestamp import java.util.UUID -class IcebergDocumentSpec extends AnyFlatSpec with BeforeAndAfterEach { +class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { // Define Amber Schema val amberSchema: Schema = Schema( @@ -36,29 +35,28 @@ class IcebergDocumentSpec extends AnyFlatSpec with BeforeAndAfterEach { val deserde: (IcebergSchema, Record) => Tuple = (schema, record) => IcebergUtil.fromRecord(record, amberSchema) - // Table name (unique for each test) - var tableName: String = _ - - var tableCatalog: Catalog = IcebergUtil.createJdbcCatalog( - "iceberg document test", + // Create catalog instance + val catalog: Catalog = IcebergUtil.createJdbcCatalog( + "iceberg_document_test", StorageConfig.fileStorageDirectoryUri, StorageConfig.jdbcUrl, StorageConfig.jdbcUsername, StorageConfig.jdbcPassword ) - val tableNamespace = "test" - - // IcebergDocument instance - var icebergDocument: IcebergDocument[Tuple] = _ + val tableNamespace = "test_namespace" + var tableName: String = _ override def beforeEach(): Unit = { // Generate a unique table name for each test tableName = s"test_table_${UUID.randomUUID().toString.replace("-", "")}" + super.beforeEach() + } - // Initialize IcebergDocument - icebergDocument = new IcebergDocument[Tuple]( - tableCatalog, + // Implementation of getDocument + override def getDocument: IcebergDocument[Tuple] = { + new IcebergDocument[Tuple]( + catalog, tableNamespace, tableName, icebergSchema, @@ -67,73 +65,29 @@ class IcebergDocumentSpec extends AnyFlatSpec with BeforeAndAfterEach { ) } - "IcebergDocument" should "write and read tuples successfully" in { - val tuple1 = Tuple - .builder(amberSchema) - .add("id", AttributeType.LONG, 1L) - .add("name", AttributeType.STRING, "Alice") - .add("score", AttributeType.DOUBLE, 95.5) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) - .build() - - val tuple2 = Tuple - .builder(amberSchema) - .add("id", AttributeType.LONG, 2L) - .add("name", AttributeType.STRING, "Bob") - .add("score", AttributeType.DOUBLE, 88.0) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) - .build() - - val tuple3 = Tuple - .builder(amberSchema) - .add("id", AttributeType.LONG, 3L) - .add("name", AttributeType.STRING, "John") - .add("score", AttributeType.DOUBLE, 75.5) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) - .build() - - val tuple4 = Tuple - .builder(amberSchema) - .add("id", AttributeType.LONG, 4L) - .add("name", AttributeType.STRING, "Bob") - .add("score", AttributeType.DOUBLE, 80.0) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) - .build() - - // Get writer and write tuples - var writer = icebergDocument.writer() - writer.open() - writer.putOne(tuple1) - writer.putOne(tuple2) - writer.close() - - // Read tuples back - var retrievedTuples = icebergDocument.get().toList - - assert(retrievedTuples.contains(tuple1)) - assert(retrievedTuples.contains(tuple2)) - assert(retrievedTuples.size == 2) - - // Get writer and write tuples - writer = icebergDocument.writer() - writer.open() - writer.putOne(tuple3) - writer.putOne(tuple4) - writer.close() - - retrievedTuples = icebergDocument.get().toList - - assert(retrievedTuples.contains(tuple1)) - assert(retrievedTuples.contains(tuple2)) - assert(retrievedTuples.contains(tuple3)) - assert(retrievedTuples.contains(tuple4)) - assert(retrievedTuples.size == 4) - - icebergDocument.clear() + // Implementation of isDocumentCleared + override def isDocumentCleared: Boolean = { + val identifier = TableIdentifier.of(tableNamespace, tableName) + !catalog.tableExists(identifier) } - it should "handle empty reads gracefully" in { - val retrievedTuples = icebergDocument.get().toList - assert(retrievedTuples.isEmpty) + // Implementation of generateSampleItems + override def generateSampleItems(): List[Tuple] = { + List( + Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 1L) + .add("name", AttributeType.STRING, "Alice") + .add("score", AttributeType.DOUBLE, 95.5) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build(), + Tuple + .builder(amberSchema) + .add("id", AttributeType.LONG, 2L) + .add("name", AttributeType.STRING, "Bob") + .add("score", AttributeType.DOUBLE, 88.0) + .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .build() + ) } } From 4663fef3b95c63c29b3e979149207d2fc96e6373 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 19 Dec 2024 12:11:08 -0800 Subject: [PATCH 16/49] finish 1st viable version --- .../result/iceberg/IcebergDocument.scala | 78 ++++++++++++------- .../result/iceberg/IcebergTableWriter.scala | 4 +- .../edu/uci/ics/amber/util/IcebergUtil.scala | 41 ++++++---- .../storage/model/VirtualDocumentSpec.scala | 36 ++++++++- .../result/iceberg/IcebergDocumentSpec.scala | 58 ++++++++++---- 5 files changed, 157 insertions(+), 60 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 96dbaa51800..571bd53c348 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -2,9 +2,10 @@ package edu.uci.ics.amber.core.storage.result.iceberg import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} import edu.uci.ics.amber.util.IcebergUtil -import org.apache.iceberg.Table +import org.apache.iceberg.{Snapshot, Table} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} import org.apache.iceberg.data.{IcebergGenerics, Record} +import org.apache.iceberg.exceptions.NoSuchTableException import org.apache.iceberg.io.CloseableIterable import java.net.URI @@ -26,7 +27,11 @@ class IcebergDocument[T >: Null <: AnyRef]( * Returns the URI of the table location. */ override def getURI: URI = { - val table = IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) + val table = IcebergUtil + .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) + .getOrElse( + throw new NoSuchTableException(f"table ${tableNamespace}.${tableName} doesn't exist") + ) URI.create(table.location()) } @@ -41,36 +46,45 @@ class IcebergDocument[T >: Null <: AnyRef]( } } - /** - * Returns an iterator that iterates over all records in the table, including new records - * from concurrent writers as they commit. - */ - override def get(): Iterator[T] = + override def get(): Iterator[T] = { new Iterator[T] { - private val table = - IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) - private var currentSnapshot = table.currentSnapshot() - private var recordIterator = loadRecords() + private var table: Option[Table] = loadTable() + private var currentSnapshot: Option[Snapshot] = + table.flatMap(t => Option(t.currentSnapshot())) + private var recordIterator: Iterator[T] = loadRecords() + + /** + * Loads the table, handling cases where it may not exist. + */ + private def loadTable(): Option[Table] = { + IcebergUtil.loadTable( + catalog, + tableNamespace, + tableName, + tableSchema, + createIfNotExist = false + ) + } /** * Loads all records from the current snapshot. */ private def loadRecords(): Iterator[T] = { - if (currentSnapshot != null) { - try { - val records: CloseableIterable[Record] = IcebergGenerics.read(table).build() - records.iterator().asScala.map(record => deserde(tableSchema, record)) - } catch { - case _: java.io.FileNotFoundException => - println("Metadata file not found. Returning an empty iterator.") - Iterator.empty - case e: Exception => - println(s"Error during record loading: ${e.getMessage}") - e.printStackTrace() - Iterator.empty - } - } else { - Iterator.empty + table match { + case Some(t) if currentSnapshot.isDefined => + try { + val records: CloseableIterable[Record] = IcebergGenerics.read(t).build() + records.iterator().asScala.map(record => deserde(tableSchema, record)) + } catch { + case _: java.io.FileNotFoundException => + println("Metadata file not found. Returning an empty iterator.") + Iterator.empty + case e: Exception => + println(s"Error during record loading: ${e.getMessage}") + e.printStackTrace() + Iterator.empty + } + case _ => Iterator.empty } } @@ -79,8 +93,10 @@ class IcebergDocument[T >: Null <: AnyRef]( true } else { // Refresh the table and check for new commits - table.refresh() - val newSnapshot = table.currentSnapshot() + table = loadTable() + table.foreach(_.refresh()) + val newSnapshot = table.flatMap(t => Option(t.currentSnapshot())) + if (newSnapshot != currentSnapshot) { currentSnapshot = newSnapshot recordIterator = loadRecords() @@ -91,8 +107,12 @@ class IcebergDocument[T >: Null <: AnyRef]( } } - override def next(): T = recordIterator.next() + override def next(): T = { + if (!hasNext) throw new NoSuchElementException("No more records available") + recordIterator.next() + } } + } /** * Returns a BufferedItemWriter for writing data to the table. diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 5c8f27ecc06..462fe87d7ae 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -27,7 +27,9 @@ class IcebergTableWriter[T]( // Load the Iceberg table private val table: Table = - IcebergUtil.loadOrCreateTable(catalog, tableNamespace, tableName, tableSchema) + IcebergUtil + .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = true) + .get override def open(): Unit = withLock { diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 4927c641a03..3887917118e 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -11,6 +11,7 @@ import org.apache.iceberg.types.Type.PrimitiveType import org.apache.iceberg.{CatalogProperties, Table, Schema => IcebergSchema} import java.net.URI +import java.nio.ByteBuffer import java.sql.Timestamp import java.time.LocalDateTime import java.time.ZoneId @@ -22,8 +23,8 @@ object IcebergUtil { * Creates and initializes a JdbcCatalog with the given parameters. * * @param catalogName The name of the catalog. - * @param warehouseUri The warehouse directory path. - * @param jdbcUri The JDBC URI for the catalog. + * @param warehouseUri The warehouse directory path. + * @param jdbcUri The JDBC URI for the catalog. * @param jdbcUser The JDBC username. * @param jdbcPassword The JDBC password. * @return The initialized JdbcCatalog. @@ -49,17 +50,21 @@ object IcebergUtil { catalog } - def loadOrCreateTable( + def loadTable( catalog: Catalog, tableNamespace: String, tableName: String, - tableSchema: IcebergSchema - ): Table = { + tableSchema: IcebergSchema, + createIfNotExist: Boolean + ): Option[Table] = { val identifier = TableIdentifier.of(tableNamespace, tableName) if (!catalog.tableExists(identifier)) { - catalog.createTable(identifier, tableSchema) + if (!createIfNotExist) { + return None + } + Some(catalog.createTable(identifier, tableSchema)) } else { - catalog.loadTable(identifier) + Some(catalog.loadTable(identifier)) } } @@ -72,7 +77,7 @@ object IcebergUtil { def toIcebergSchema(amberSchema: Schema): IcebergSchema = { val icebergFields = amberSchema.getAttributes.zipWithIndex.map { case (attribute, index) => - Types.NestedField.required(index + 1, attribute.getName, toIcebergType(attribute.getType)) + Types.NestedField.optional(index + 1, attribute.getName, toIcebergType(attribute.getType)) } new IcebergSchema(icebergFields.asJava) } @@ -98,21 +103,22 @@ object IcebergUtil { } /** - * Converts a custom Amber `Tuple` to an Iceberg `GenericRecord`. + * Converts a custom Amber `Tuple` to an Iceberg `GenericRecord`, handling `null` values. * * @param tuple The custom Amber Tuple. * @return An Iceberg GenericRecord. */ def toGenericRecord(tuple: Tuple): Record = { - // Convert the Amber schema to an Iceberg schema val icebergSchema = toIcebergSchema(tuple.schema) val record = GenericRecord.create(icebergSchema) tuple.schema.getAttributes.zipWithIndex.foreach { case (attribute, index) => val value = tuple.getField[AnyRef](index) match { - case ts: Timestamp => ts.toInstant.atZone(ZoneId.systemDefault()).toLocalDateTime - case other => other + case null => null + case ts: Timestamp => ts.toInstant.atZone(ZoneId.systemDefault()).toLocalDateTime + case bytes: Array[Byte] => ByteBuffer.wrap(bytes) + case other => other } record.setField(attribute.getName, value) } @@ -121,17 +127,22 @@ object IcebergUtil { } /** - * Converts an Iceberg `Record` to an Amber `Tuple`. + * Converts an Iceberg `Record` to an Amber `Tuple`, handling `null` values. * - * @param record The Iceberg Record. + * @param record The Iceberg Record. * @param amberSchema The corresponding Amber Schema. * @return An Amber Tuple. */ def fromRecord(record: Record, amberSchema: Schema): Tuple = { val fieldValues = amberSchema.getAttributes.map { attribute => val value = record.getField(attribute.getName) match { + case null => null case ldt: LocalDateTime => Timestamp.valueOf(ldt) - case other => other + case buffer: ByteBuffer => + val bytes = new Array[Byte](buffer.remaining()) + buffer.get(bytes) + bytes + case other => other } value } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 845aa164414..9dd13e49947 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -42,9 +42,43 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // Read items back val retrievedItems = document.get().toList - assert(retrievedItems == items, "The retrieved items should match the written items.") + assert(retrievedItems == items) } + "VirtualDocument" should "read items while writer is writing new data" in { + val allItems = generateSampleItems() + + // Split the items into two batches + val (batch1, batch2) = allItems.splitAt(allItems.length / 2) + + // Create a reader before any data is written + val reader = document.get() + assert(!reader.hasNext, "Reader should initially have no data.") + + // Write the first batch + val writer = document.writer() + writer.open() + batch1.foreach(writer.putOne) + writer.close() + + // The reader should detect and read the first batch + val retrievedBatch1 = reader.take(batch1.length).toList + assert(retrievedBatch1.toSet == batch1.toSet, "Reader should read the first batch correctly.") + + // Write the second batch + val writer2 = document.writer() + writer2.open() + batch2.foreach(writer2.putOne) + writer2.close() + + // The reader should detect and read the second batch + val retrievedBatch2 = reader.toList + assert(retrievedBatch2.toSet == batch2.toSet, "Reader should read the second batch correctly.") + + // Verify that the combined retrieved items match the original items + val retrievedItems = retrievedBatch1 ++ retrievedBatch2 + assert(retrievedItems.toSet == allItems.toSet, "Reader should read all items correctly.") + } it should "clear the document" in { val items = generateSampleItems() diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index c84f339a235..8195c798ab9 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -15,13 +15,16 @@ import java.util.UUID class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { - // Define Amber Schema + // Define Amber Schema with all possible attribute types val amberSchema: Schema = Schema( List( - new Attribute("id", AttributeType.LONG), - new Attribute("name", AttributeType.STRING), - new Attribute("score", AttributeType.DOUBLE), - new Attribute("timestamp", AttributeType.TIMESTAMP) + new Attribute("col-string", AttributeType.STRING), + new Attribute("col-int", AttributeType.INTEGER), + new Attribute("col-bool", AttributeType.BOOLEAN), + new Attribute("col-long", AttributeType.LONG), + new Attribute("col-double", AttributeType.DOUBLE), + new Attribute("col-timestamp", AttributeType.TIMESTAMP) +// new Attribute("col-binary", AttributeType.BINARY) ) ) @@ -65,7 +68,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { ) } - // Implementation of isDocumentCleared + // Implementation of isDocumentClearedgetSam override def isDocumentCleared: Boolean = { val identifier = TableIdentifier.of(tableNamespace, tableName) !catalog.tableExists(identifier) @@ -76,18 +79,45 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { List( Tuple .builder(amberSchema) - .add("id", AttributeType.LONG, 1L) - .add("name", AttributeType.STRING, "Alice") - .add("score", AttributeType.DOUBLE, 95.5) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .add("col-string", AttributeType.STRING, "Hello World") + .add("col-int", AttributeType.INTEGER, 42) + .add("col-bool", AttributeType.BOOLEAN, true) + .add("col-long", AttributeType.LONG, 12345678901234L) + .add("col-double", AttributeType.DOUBLE, 3.14159) + .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) +// .add("col-binary", AttributeType.BINARY, Array[Byte](1, 2, 3, 4, 5)) .build(), Tuple .builder(amberSchema) - .add("id", AttributeType.LONG, 2L) - .add("name", AttributeType.STRING, "Bob") - .add("score", AttributeType.DOUBLE, 88.0) - .add("timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .add("col-string", AttributeType.STRING, "") + .add("col-int", AttributeType.INTEGER, -1) + .add("col-bool", AttributeType.BOOLEAN, false) + .add("col-long", AttributeType.LONG, -98765432109876L) + .add("col-double", AttributeType.DOUBLE, -0.001) + .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(0L)) +// .add("col-binary", AttributeType.BINARY, Array[Byte]()) .build() +// +// Tuple +// .builder(amberSchema) +// .add("col-string", AttributeType.STRING, "Special Characters: \n\t\r") +// .add("col-int", AttributeType.INTEGER, Int.MaxValue) +// .add("col-bool", AttributeType.BOOLEAN, true) +// .add("col-long", AttributeType.LONG, Long.MaxValue) +// .add("col-double", AttributeType.DOUBLE, Double.MaxValue) +// .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(1234567890L)) +//// .add("col-binary", AttributeType.BINARY, Array.fill[Byte](1000)('a')) +// .build(), +// Tuple +// .builder(amberSchema) +// .add("col-string", AttributeType.STRING, null) +// .add("col-int", AttributeType.INTEGER, null) +// .add("col-bool", AttributeType.BOOLEAN, null) +// .add("col-long", AttributeType.LONG, null) +// .add("col-double", AttributeType.DOUBLE, null) +// .add("col-timestamp", AttributeType.TIMESTAMP, null) +//// .add("col-binary", AttributeType.BINARY, null) +// .build() ) } } From 9607f9898ec792f58326a5571c95b99649c46b9d Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 19 Dec 2024 13:49:16 -0800 Subject: [PATCH 17/49] fix the append read --- .../result/iceberg/IcebergDocument.scala | 54 ++++++++++--------- 1 file changed, 30 insertions(+), 24 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 571bd53c348..b8d6af6f6f3 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -49,32 +49,44 @@ class IcebergDocument[T >: Null <: AnyRef]( override def get(): Iterator[T] = { new Iterator[T] { private var table: Option[Table] = loadTable() - private var currentSnapshot: Option[Snapshot] = - table.flatMap(t => Option(t.currentSnapshot())) + private var lastSnapshotId: Option[Long] = None private var recordIterator: Iterator[T] = loadRecords() /** - * Loads the table, handling cases where it may not exist. - */ + * Loads the table, handling cases where it may not exist. + */ private def loadTable(): Option[Table] = { - IcebergUtil.loadTable( - catalog, - tableNamespace, - tableName, - tableSchema, - createIfNotExist = false - ) + IcebergUtil.loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) } /** - * Loads all records from the current snapshot. - */ + * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. + */ private def loadRecords(): Iterator[T] = { table match { - case Some(t) if currentSnapshot.isDefined => + case Some(t) => try { - val records: CloseableIterable[Record] = IcebergGenerics.read(t).build() + val currentSnapshot = Option(t.currentSnapshot()) + val currentSnapshotId = currentSnapshot.map(_.snapshotId()) + + val records: CloseableIterable[Record] = (lastSnapshotId, currentSnapshotId) match { + case (Some(lastId), Some(currId)) if lastId != currId => + // Perform incremental append scan if snapshot IDs are different + IcebergGenerics.read(t).appendsAfter(lastId).build() + + case (None, Some(_)) => + // First read, perform a full scan + IcebergGenerics.read(t).build() + + case _ => + // No new data; return an empty iterator + CloseableIterable.empty() + } + + // Update the last snapshot ID to the current one + lastSnapshotId = currentSnapshotId records.iterator().asScala.map(record => deserde(tableSchema, record)) + } catch { case _: java.io.FileNotFoundException => println("Metadata file not found. Returning an empty iterator.") @@ -84,6 +96,7 @@ class IcebergDocument[T >: Null <: AnyRef]( e.printStackTrace() Iterator.empty } + case _ => Iterator.empty } } @@ -95,15 +108,8 @@ class IcebergDocument[T >: Null <: AnyRef]( // Refresh the table and check for new commits table = loadTable() table.foreach(_.refresh()) - val newSnapshot = table.flatMap(t => Option(t.currentSnapshot())) - - if (newSnapshot != currentSnapshot) { - currentSnapshot = newSnapshot - recordIterator = loadRecords() - recordIterator.hasNext - } else { - false - } + recordIterator = loadRecords() + recordIterator.hasNext } } From d2d0ed79b187480d88ac47f7128151eef0dfacd0 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Thu, 19 Dec 2024 23:18:12 -0800 Subject: [PATCH 18/49] finish append read --- .../result/iceberg/IcebergDocument.scala | 16 +++++--- .../storage/model/VirtualDocumentSpec.scala | 6 +-- .../result/iceberg/IcebergDocumentSpec.scala | 41 +++++++++---------- 3 files changed, 34 insertions(+), 29 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index b8d6af6f6f3..b41ae5c30f5 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -53,15 +53,21 @@ class IcebergDocument[T >: Null <: AnyRef]( private var recordIterator: Iterator[T] = loadRecords() /** - * Loads the table, handling cases where it may not exist. - */ + * Loads the table, handling cases where it may not exist. + */ private def loadTable(): Option[Table] = { - IcebergUtil.loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) + IcebergUtil.loadTable( + catalog, + tableNamespace, + tableName, + tableSchema, + createIfNotExist = false + ) } /** - * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. - */ + * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. + */ private def loadRecords(): Iterator[T] = { table match { case Some(t) => diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 9dd13e49947..f367fe53fae 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -63,7 +63,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // The reader should detect and read the first batch val retrievedBatch1 = reader.take(batch1.length).toList - assert(retrievedBatch1.toSet == batch1.toSet, "Reader should read the first batch correctly.") + assert(retrievedBatch1 == batch1, "Reader should read the first batch correctly.") // Write the second batch val writer2 = document.writer() @@ -73,11 +73,11 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // The reader should detect and read the second batch val retrievedBatch2 = reader.toList - assert(retrievedBatch2.toSet == batch2.toSet, "Reader should read the second batch correctly.") + assert(retrievedBatch2 == batch2, "Reader should read the second batch correctly.") // Verify that the combined retrieved items match the original items val retrievedItems = retrievedBatch1 ++ retrievedBatch2 - assert(retrievedItems.toSet == allItems.toSet, "Reader should read all items correctly.") + assert(retrievedItems == allItems, "Reader should read all items correctly.") } it should "clear the document" in { val items = generateSampleItems() diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 8195c798ab9..59391800401 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -96,28 +96,27 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-double", AttributeType.DOUBLE, -0.001) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(0L)) // .add("col-binary", AttributeType.BINARY, Array[Byte]()) + .build(), + Tuple + .builder(amberSchema) + .add("col-string", AttributeType.STRING, "Special Characters: \n\t\r") + .add("col-int", AttributeType.INTEGER, Int.MaxValue) + .add("col-bool", AttributeType.BOOLEAN, true) + .add("col-long", AttributeType.LONG, Long.MaxValue) + .add("col-double", AttributeType.DOUBLE, Double.MaxValue) + .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(1234567890L)) +// .add("col-binary", AttributeType.BINARY, Array.fill[Byte](1000)('a')) + .build(), + Tuple + .builder(amberSchema) + .add("col-string", AttributeType.STRING, null) + .add("col-int", AttributeType.INTEGER, null) + .add("col-bool", AttributeType.BOOLEAN, null) + .add("col-long", AttributeType.LONG, null) + .add("col-double", AttributeType.DOUBLE, null) + .add("col-timestamp", AttributeType.TIMESTAMP, null) +// .add("col-binary", AttributeType.BINARY, null) .build() -// -// Tuple -// .builder(amberSchema) -// .add("col-string", AttributeType.STRING, "Special Characters: \n\t\r") -// .add("col-int", AttributeType.INTEGER, Int.MaxValue) -// .add("col-bool", AttributeType.BOOLEAN, true) -// .add("col-long", AttributeType.LONG, Long.MaxValue) -// .add("col-double", AttributeType.DOUBLE, Double.MaxValue) -// .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(1234567890L)) -//// .add("col-binary", AttributeType.BINARY, Array.fill[Byte](1000)('a')) -// .build(), -// Tuple -// .builder(amberSchema) -// .add("col-string", AttributeType.STRING, null) -// .add("col-int", AttributeType.INTEGER, null) -// .add("col-bool", AttributeType.BOOLEAN, null) -// .add("col-long", AttributeType.LONG, null) -// .add("col-double", AttributeType.DOUBLE, null) -// .add("col-timestamp", AttributeType.TIMESTAMP, null) -//// .add("col-binary", AttributeType.BINARY, null) -// .build() ) } } From f4ea0e30673ff5e92c4a277ecce73a39d9c06f46 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 10:13:34 -0800 Subject: [PATCH 19/49] finish concurrent write test --- .../result/iceberg/IcebergDocument.scala | 151 ++++++++---------- .../result/iceberg/IcebergTableWriter.scala | 77 ++++----- .../amber/core/storage/util/StorageUtil.scala | 23 +++ .../edu/uci/ics/amber/util/IcebergUtil.scala | 15 +- .../storage/model/VirtualDocumentSpec.scala | 57 ++++++- .../result/iceberg/IcebergDocumentSpec.scala | 34 ++-- 6 files changed, 209 insertions(+), 148 deletions(-) create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/util/StorageUtil.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index b41ae5c30f5..51705e77a62 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -1,6 +1,7 @@ package edu.uci.ics.amber.core.storage.result.iceberg import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} +import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock, withWriteLock} import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.{Snapshot, Table} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} @@ -9,7 +10,7 @@ import org.apache.iceberg.exceptions.NoSuchTableException import org.apache.iceberg.io.CloseableIterable import java.net.URI -import java.util.concurrent.locks.ReentrantReadWriteLock +import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock} import scala.jdk.CollectionConverters._ class IcebergDocument[T >: Null <: AnyRef]( @@ -22,6 +23,11 @@ class IcebergDocument[T >: Null <: AnyRef]( ) extends VirtualDocument[T] { private val lock = new ReentrantReadWriteLock() + synchronized { + IcebergUtil + .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = true) + .get + } /** * Returns the URI of the table location. @@ -39,92 +45,84 @@ class IcebergDocument[T >: Null <: AnyRef]( * Deletes the table. */ override def clear(): Unit = - withWriteLock { + withWriteLock(lock) { val identifier = TableIdentifier.of(tableNamespace, tableName) if (catalog.tableExists(identifier)) { catalog.dropTable(identifier) } } - override def get(): Iterator[T] = { - new Iterator[T] { - private var table: Option[Table] = loadTable() - private var lastSnapshotId: Option[Long] = None - private var recordIterator: Iterator[T] = loadRecords() - - /** - * Loads the table, handling cases where it may not exist. - */ - private def loadTable(): Option[Table] = { - IcebergUtil.loadTable( - catalog, - tableNamespace, - tableName, - tableSchema, - createIfNotExist = false - ) - } + override def get(): Iterator[T] = + withReadLock(lock) { + new Iterator[T] { + private val iteLock = new ReentrantLock() + private var table: Option[Table] = loadTable() + private var lastSnapshotId: Option[Long] = None + private var recordIterator: Iterator[T] = loadRecords() + + private def loadTable(): Option[Table] = { + IcebergUtil.loadTable( + catalog, + tableNamespace, + tableName, + tableSchema, + createIfNotExist = false + ) + } - /** - * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. - */ - private def loadRecords(): Iterator[T] = { - table match { - case Some(t) => - try { - val currentSnapshot = Option(t.currentSnapshot()) - val currentSnapshotId = currentSnapshot.map(_.snapshotId()) - - val records: CloseableIterable[Record] = (lastSnapshotId, currentSnapshotId) match { - case (Some(lastId), Some(currId)) if lastId != currId => - // Perform incremental append scan if snapshot IDs are different - IcebergGenerics.read(t).appendsAfter(lastId).build() - - case (None, Some(_)) => - // First read, perform a full scan - IcebergGenerics.read(t).build() - - case _ => - // No new data; return an empty iterator - CloseableIterable.empty() - } - - // Update the last snapshot ID to the current one - lastSnapshotId = currentSnapshotId - records.iterator().asScala.map(record => deserde(tableSchema, record)) - - } catch { - case _: java.io.FileNotFoundException => - println("Metadata file not found. Returning an empty iterator.") - Iterator.empty - case e: Exception => - println(s"Error during record loading: ${e.getMessage}") - e.printStackTrace() - Iterator.empty + /** + * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. + */ + private def loadRecords(): Iterator[T] = + withLock(iteLock) { + table match { + case Some(t) => + val currentSnapshot = Option(t.currentSnapshot()) + val currentSnapshotId = currentSnapshot.map(_.snapshotId()) + + val records: CloseableIterable[Record] = (lastSnapshotId, currentSnapshotId) match { + case (Some(lastId), Some(currId)) if lastId != currId => + // Perform incremental append scan if snapshot IDs are different + IcebergGenerics.read(t).appendsAfter(lastId).build() + + case (None, Some(_)) => + // First read, perform a full scan + IcebergGenerics.read(t).build() + + case _ => + // No new data; return an empty iterator + CloseableIterable.empty() + } + + // Update the last snapshot ID to the current one + lastSnapshotId = currentSnapshotId + records.iterator().asScala.map(record => deserde(tableSchema, record)) + + case _ => Iterator.empty } + } + + override def hasNext: Boolean = { + if (recordIterator.hasNext) { + true + } else { + // Refresh the table and check for new commits + if (table.isEmpty) { + table = loadTable() + } + table.foreach(_.refresh()) + recordIterator = loadRecords() + recordIterator.hasNext - case _ => Iterator.empty + } } - } - override def hasNext: Boolean = { - if (recordIterator.hasNext) { - true - } else { - // Refresh the table and check for new commits - table = loadTable() - table.foreach(_.refresh()) - recordIterator = loadRecords() - recordIterator.hasNext + override def next(): T = { + if (!hasNext) throw new NoSuchElementException("No more records available") + recordIterator.next() } } - - override def next(): T = { - if (!hasNext) throw new NoSuchElementException("No more records available") - recordIterator.next() - } } - } /** * Returns a BufferedItemWriter for writing data to the table. @@ -132,11 +130,4 @@ class IcebergDocument[T >: Null <: AnyRef]( override def writer(): BufferedItemWriter[T] = { new IcebergTableWriter[T](catalog, tableNamespace, tableName, tableSchema, serde) } - - // Utility function to wrap code block with write lock - private def withWriteLock[M](block: => M): M = { - lock.writeLock().lock() - try block - finally lock.writeLock().unlock() - } } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 462fe87d7ae..4802e0001cd 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -1,6 +1,7 @@ package edu.uci.ics.amber.core.storage.result.iceberg import edu.uci.ics.amber.core.storage.model.BufferedItemWriter +import edu.uci.ics.amber.core.storage.util.StorageUtil.withLock import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.{Schema, Table} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} @@ -23,21 +24,22 @@ class IcebergTableWriter[T]( private val lock = new ReentrantLock() private val buffer = new ArrayBuffer[T]() - override val bufferSize: Int = 1024 + override val bufferSize: Int = 3000 // Load the Iceberg table - private val table: Table = + private val table: Table = synchronized { IcebergUtil - .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = true) + .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) .get + } override def open(): Unit = - withLock { + withLock(lock) { buffer.clear() } override def putOne(item: T): Unit = - withLock { + withLock(lock) { buffer.append(item) if (buffer.size >= bufferSize) { flushBuffer() @@ -45,61 +47,46 @@ class IcebergTableWriter[T]( } override def removeOne(item: T): Unit = - withLock { + withLock(lock) { buffer -= item } private def flushBuffer(): Unit = - withLock { + withLock(lock) { if (buffer.nonEmpty) { - try { - // Create a unique file path using UUID - val filepath = s"${table.location()}/${UUID.randomUUID().toString}" - val outputFile: OutputFile = table.io().newOutputFile(filepath) - - // Create a Parquet data writer - val dataWriter: DataWriter[Record] = Parquet - .writeData(outputFile) - .forTable(table) - .createWriterFunc(GenericParquetWriter.buildWriter) - .overwrite() - .build() - - try { - buffer.foreach { item => - val record = serde(item) - dataWriter.write(record) - } - } finally { - dataWriter.close() - } - // Commit the new file to the table - val dataFile = dataWriter.toDataFile - table.newAppend().appendFile(dataFile).commit() + // Create a unique file path using UUID + val filepath = s"${table.location()}/${UUID.randomUUID().toString}" + val outputFile: OutputFile = table.io().newOutputFile(filepath) - println(s"Flushed ${buffer.size} records to ${filepath}") + // Create a Parquet data writer + val dataWriter: DataWriter[Record] = Parquet + .writeData(outputFile) + .forTable(table) + .createWriterFunc(GenericParquetWriter.buildWriter) + .overwrite() + .build() - buffer.clear() - } catch { - case e: Exception => - println(s"Error during flush: ${e.getMessage}") - e.printStackTrace() + try { + buffer.foreach { item => + val record = serde(item) + dataWriter.write(record) + } + } finally { + dataWriter.close() } + + // Commit the new file to the table + val dataFile = dataWriter.toDataFile + table.newAppend().appendFile(dataFile).commit() + buffer.clear() } } override def close(): Unit = - withLock { + withLock(lock) { if (buffer.nonEmpty) { flushBuffer() } } - - // Utility function to wrap code block with write lock - private def withLock[M](block: => M): M = { - lock.lock() - try block - finally lock.unlock() - } } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/util/StorageUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/util/StorageUtil.scala new file mode 100644 index 00000000000..0e606015e8b --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/util/StorageUtil.scala @@ -0,0 +1,23 @@ +package edu.uci.ics.amber.core.storage.util + +import java.util.concurrent.locks.{Lock, ReadWriteLock} + +object StorageUtil { + def withWriteLock[M](rwLock: ReadWriteLock)(block: => M): M = { + rwLock.writeLock().lock() + try block + finally rwLock.writeLock().unlock() + } + + def withReadLock[M](rwLock: ReadWriteLock)(block: => M): M = { + rwLock.readLock().lock() + try block + finally rwLock.readLock().unlock() + } + + def withLock[M](lock: Lock)(block: => M): M = { + lock.lock() + try block + finally lock.unlock() + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 3887917118e..273a883416a 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -8,7 +8,7 @@ import org.apache.iceberg.types.Types import org.apache.iceberg.data.{GenericRecord, Record} import org.apache.iceberg.jdbc.JdbcCatalog import org.apache.iceberg.types.Type.PrimitiveType -import org.apache.iceberg.{CatalogProperties, Table, Schema => IcebergSchema} +import org.apache.iceberg.{CatalogProperties, PartitionSpec, Table, Schema => IcebergSchema} import java.net.URI import java.nio.ByteBuffer @@ -57,12 +57,23 @@ object IcebergUtil { tableSchema: IcebergSchema, createIfNotExist: Boolean ): Option[Table] = { + val tableProperties = Map( + "commit.retry.num-retries" -> "10", + "commit.retry.min-wait-ms" -> "10" + ) val identifier = TableIdentifier.of(tableNamespace, tableName) if (!catalog.tableExists(identifier)) { if (!createIfNotExist) { return None } - Some(catalog.createTable(identifier, tableSchema)) + Some( + catalog.createTable( + identifier, + tableSchema, + PartitionSpec.unpartitioned, + tableProperties.asJava + ) + ) } else { Some(catalog.loadTable(identifier)) } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index f367fe53fae..0c9b27be99c 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -2,6 +2,10 @@ package edu.uci.ics.amber.core.storage.model import org.scalatest.BeforeAndAfterEach import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.time.SpanSugar.convertIntToGrainOfTime + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.{Await, Future} /** * A trait for testing VirtualDocument implementations. @@ -42,7 +46,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // Read items back val retrievedItems = document.get().toList - assert(retrievedItems == items) + assert(retrievedItems.toSet == items.toSet) } "VirtualDocument" should "read items while writer is writing new data" in { @@ -63,7 +67,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // The reader should detect and read the first batch val retrievedBatch1 = reader.take(batch1.length).toList - assert(retrievedBatch1 == batch1, "Reader should read the first batch correctly.") + assert(retrievedBatch1.toSet == batch1.toSet, "Reader should read the first batch correctly.") // Write the second batch val writer2 = document.writer() @@ -73,11 +77,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // The reader should detect and read the second batch val retrievedBatch2 = reader.toList - assert(retrievedBatch2 == batch2, "Reader should read the second batch correctly.") - - // Verify that the combined retrieved items match the original items - val retrievedItems = retrievedBatch1 ++ retrievedBatch2 - assert(retrievedItems == allItems, "Reader should read all items correctly.") + assert(retrievedBatch2.toSet == batch2.toSet, "Reader should read the second batch correctly.") } it should "clear the document" in { val items = generateSampleItems() @@ -104,6 +104,49 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { assert(retrievedItems.isEmpty, "Reading from an empty document should return an empty list.") } + it should "handle concurrent writes and read all items correctly" in { + val allItems = generateSampleItems() + val numWriters = 10 + + // Calculate the batch size and the remainder + val batchSize = allItems.length / numWriters + val remainder = allItems.length % numWriters + + // Create batches using a simple for loop + val itemBatches = (0 until numWriters).map { i => + val start = i * batchSize + Math.min(i, remainder) + val end = start + batchSize + (if (i < remainder) 1 else 0) + allItems.slice(start, end) + }.toList + + assert( + itemBatches.length == numWriters, + s"Expected $numWriters batches but got ${itemBatches.length}" + ) + + // Perform concurrent writes + val writeFutures = itemBatches.map { batch => + Future { + val writer = document.writer() + writer.open() + batch.foreach(writer.putOne) + writer.close() + } + } + + // Wait for all writers to complete + Await.result(Future.sequence(writeFutures), 30.seconds) + + // Read all items back + val retrievedItems = document.get().toList + + // Verify that the retrieved items match the original items + assert( + retrievedItems.toSet == allItems.toSet, + "All items should be read correctly after concurrent writes." + ) + } + /** * Generates a sample list of items for testing. * Subclasses should override this to provide their specific sample items. diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 59391800401..8db2d87cc5d 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -74,9 +74,8 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { !catalog.tableExists(identifier) } - // Implementation of generateSampleItems override def generateSampleItems(): List[Tuple] = { - List( + val baseTuples = List( Tuple .builder(amberSchema) .add("col-string", AttributeType.STRING, "Hello World") @@ -85,7 +84,6 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, 12345678901234L) .add("col-double", AttributeType.DOUBLE, 3.14159) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) -// .add("col-binary", AttributeType.BINARY, Array[Byte](1, 2, 3, 4, 5)) .build(), Tuple .builder(amberSchema) @@ -95,7 +93,6 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, -98765432109876L) .add("col-double", AttributeType.DOUBLE, -0.001) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(0L)) -// .add("col-binary", AttributeType.BINARY, Array[Byte]()) .build(), Tuple .builder(amberSchema) @@ -105,18 +102,27 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, Long.MaxValue) .add("col-double", AttributeType.DOUBLE, Double.MaxValue) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(1234567890L)) -// .add("col-binary", AttributeType.BINARY, Array.fill[Byte](1000)('a')) - .build(), + .build() + ) + + // Generate additional tuples with random data + val additionalTuples = (1 to 20000).map { i => Tuple .builder(amberSchema) - .add("col-string", AttributeType.STRING, null) - .add("col-int", AttributeType.INTEGER, null) - .add("col-bool", AttributeType.BOOLEAN, null) - .add("col-long", AttributeType.LONG, null) - .add("col-double", AttributeType.DOUBLE, null) - .add("col-timestamp", AttributeType.TIMESTAMP, null) -// .add("col-binary", AttributeType.BINARY, null) + .add("col-string", AttributeType.STRING, s"Generated String $i") + .add("col-int", AttributeType.INTEGER, i) + .add("col-bool", AttributeType.BOOLEAN, i % 2 == 0) + .add("col-long", AttributeType.LONG, i.toLong * 1000000L) + .add("col-double", AttributeType.DOUBLE, i * 0.12345) + .add( + "col-timestamp", + AttributeType.TIMESTAMP, + new Timestamp(System.currentTimeMillis() + i * 1000L) + ) .build() - ) + } + + // Combine the base tuples with the generated tuples + baseTuples ++ additionalTuples } } From e46311cd8fe20054c30f8500c7cff99b3fadb95b Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 14:31:47 -0800 Subject: [PATCH 20/49] resolve the binary type issue --- .../edu/uci/ics/amber/core/tuple/Tuple.scala | 11 +- .../storage/model/VirtualDocumentSpec.scala | 53 +++++ .../result/iceberg/IcebergDocumentSpec.scala | 55 +++++- .../uci/ics/amber/util/IcebergUtilSpec.scala | 184 ++++++++++++++++++ 4 files changed, 292 insertions(+), 11 deletions(-) create mode 100644 core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/Tuple.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/Tuple.scala index dea62ba6c00..af6fa86fc68 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/Tuple.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/tuple/Tuple.scala @@ -73,8 +73,15 @@ case class Tuple @JsonCreator() ( override def equals(obj: Any): Boolean = obj match { - case that: Tuple => (this.getFields sameElements that.getFields) && this.schema == that.schema - case _ => false + case that: Tuple => + this.schema == that.schema && + this.getFields.zip(that.getFields).forall { + case (field1: Array[Byte], field2: Array[Byte]) => + field1.sameElements(field2) // for Binary, use sameElements instead of == to compare + case (field1, field2) => + field1 == field2 + } + case _ => false } def getPartialTuple(attributeNames: List[String]): Tuple = { diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 0c9b27be99c..9b816f31ebd 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -147,6 +147,59 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } + it should "allow a reader to read data while a writer is writing items incrementally" in { + val allItems = generateSampleItems() + val batchSize = allItems.length / 5 // Divide items into 5 incremental batches + + // Split items into 5 batches + val itemBatches = allItems.grouped(batchSize).toList + + // Flag to indicate when writing is done + @volatile var writingComplete = false + + // Start the writer in a Future to write batches with delays + val writerFuture = Future { + val writer = document.writer() + writer.open() + try { + itemBatches.foreach { batch => + batch.foreach(writer.putOne) + Thread.sleep(500) // Simulate delay between batches + } + } finally { + writer.close() + writingComplete = true + } + } + + // Start the reader in another Future + val readerFuture = Future { + val reader = document.get() + val retrievedItems = scala.collection.mutable.ListBuffer[T]() + + // Keep checking for new data until writing is complete and no more items are available + while (!writingComplete || reader.hasNext) { + if (reader.hasNext) { + retrievedItems += reader.next() + } else { + Thread.sleep(200) // Wait before retrying to avoid busy-waiting + } + } + + retrievedItems.toList + } + + // Wait for both writer and reader to complete + val retrievedItems = Await.result(readerFuture, 30.seconds) + Await.result(writerFuture, 30.seconds) + + // Verify that the retrieved items match the original items + assert( + retrievedItems.toSet == allItems.toSet, + "All items should be read correctly while writing is happening concurrently." + ) + } + /** * Generates a sample list of items for testing. * Subclasses should override this to provide their specific sample items. diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 8db2d87cc5d..88b4ee67755 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -23,8 +23,8 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { new Attribute("col-bool", AttributeType.BOOLEAN), new Attribute("col-long", AttributeType.LONG), new Attribute("col-double", AttributeType.DOUBLE), - new Attribute("col-timestamp", AttributeType.TIMESTAMP) -// new Attribute("col-binary", AttributeType.BINARY) + new Attribute("col-timestamp", AttributeType.TIMESTAMP), + new Attribute("col-binary", AttributeType.BINARY) ) ) @@ -84,6 +84,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, 12345678901234L) .add("col-double", AttributeType.DOUBLE, 3.14159) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(System.currentTimeMillis())) + .add("col-binary", AttributeType.BINARY, Array[Byte](0, 1, 2, 3, 4, 5, 6, 7)) .build(), Tuple .builder(amberSchema) @@ -93,6 +94,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, -98765432109876L) .add("col-double", AttributeType.DOUBLE, -0.001) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(0L)) + .add("col-binary", AttributeType.BINARY, Array[Byte](127, -128, 0, 64)) .build(), Tuple .builder(amberSchema) @@ -102,22 +104,57 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { .add("col-long", AttributeType.LONG, Long.MaxValue) .add("col-double", AttributeType.DOUBLE, Double.MaxValue) .add("col-timestamp", AttributeType.TIMESTAMP, new Timestamp(1234567890L)) + .add("col-binary", AttributeType.BINARY, Array[Byte](1, 2, 3, 4, 5)) .build() ) - // Generate additional tuples with random data + // Function to generate random binary data + def generateRandomBinary(size: Int): Array[Byte] = { + val array = new Array[Byte](size) + scala.util.Random.nextBytes(array) + array + } + + // Generate additional tuples with random data and occasional nulls val additionalTuples = (1 to 20000).map { i => Tuple .builder(amberSchema) - .add("col-string", AttributeType.STRING, s"Generated String $i") - .add("col-int", AttributeType.INTEGER, i) - .add("col-bool", AttributeType.BOOLEAN, i % 2 == 0) - .add("col-long", AttributeType.LONG, i.toLong * 1000000L) - .add("col-double", AttributeType.DOUBLE, i * 0.12345) + .add( + "col-string", + AttributeType.STRING, + if (i % 7 == 0) null else s"Generated String $i" + ) + .add( + "col-int", + AttributeType.INTEGER, + if (i % 5 == 0) null else i + ) + .add( + "col-bool", + AttributeType.BOOLEAN, + if (i % 6 == 0) null else i % 2 == 0 + ) + .add( + "col-long", + AttributeType.LONG, + if (i % 4 == 0) null else i.toLong * 1000000L + ) + .add( + "col-double", + AttributeType.DOUBLE, + if (i % 3 == 0) null else i * 0.12345 + ) .add( "col-timestamp", AttributeType.TIMESTAMP, - new Timestamp(System.currentTimeMillis() + i * 1000L) + if (i % 8 == 0) null + else new Timestamp(System.currentTimeMillis() + i * 1000L) + ) + .add( + "col-binary", + AttributeType.BINARY, + if (i % 9 == 0) null + else generateRandomBinary(scala.util.Random.nextInt(10) + 1) ) .build() } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala new file mode 100644 index 00000000000..9549c22a0ac --- /dev/null +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala @@ -0,0 +1,184 @@ +package edu.uci.ics.amber.util + +import edu.uci.ics.amber.core.tuple.{AttributeType, Schema, Tuple} +import org.apache.iceberg.types.Types +import org.apache.iceberg.{Schema => IcebergSchema} +import org.apache.iceberg.data.GenericRecord +import org.scalatest.flatspec.AnyFlatSpec + +import java.nio.ByteBuffer +import java.sql.Timestamp +import java.time.{LocalDateTime, ZoneId} +import scala.jdk.CollectionConverters._ + +class IcebergUtilSpec extends AnyFlatSpec { + + val texeraSchema: Schema = Schema + .builder() + .add("test-1", AttributeType.INTEGER) + .add("test-2", AttributeType.LONG) + .add("test-3", AttributeType.BOOLEAN) + .add("test-4", AttributeType.DOUBLE) + .add("test-5", AttributeType.TIMESTAMP) + .add("test-6", AttributeType.STRING) + .add("test-7", AttributeType.BINARY) + .build() + + val icebergSchema: IcebergSchema = new IcebergSchema( + List( + Types.NestedField.optional(1, "test-1", Types.IntegerType.get()), + Types.NestedField.optional(2, "test-2", Types.LongType.get()), + Types.NestedField.optional(3, "test-3", Types.BooleanType.get()), + Types.NestedField.optional(4, "test-4", Types.DoubleType.get()), + Types.NestedField.optional(5, "test-5", Types.TimestampType.withoutZone()), + Types.NestedField.optional(6, "test-6", Types.StringType.get()), + Types.NestedField.optional(7, "test-7", Types.BinaryType.get()) + ).asJava + ) + + behavior of "IcebergUtil" + + it should "convert from AttributeType to Iceberg Type correctly" in { + assert(IcebergUtil.toIcebergType(AttributeType.INTEGER) == Types.IntegerType.get()) + assert(IcebergUtil.toIcebergType(AttributeType.LONG) == Types.LongType.get()) + assert(IcebergUtil.toIcebergType(AttributeType.BOOLEAN) == Types.BooleanType.get()) + assert(IcebergUtil.toIcebergType(AttributeType.DOUBLE) == Types.DoubleType.get()) + assert(IcebergUtil.toIcebergType(AttributeType.TIMESTAMP) == Types.TimestampType.withoutZone()) + assert(IcebergUtil.toIcebergType(AttributeType.STRING) == Types.StringType.get()) + assert(IcebergUtil.toIcebergType(AttributeType.BINARY) == Types.BinaryType.get()) + } + + it should "convert from Iceberg Type to AttributeType correctly" in { + assert(IcebergUtil.fromIcebergType(Types.IntegerType.get()) == AttributeType.INTEGER) + assert(IcebergUtil.fromIcebergType(Types.LongType.get()) == AttributeType.LONG) + assert(IcebergUtil.fromIcebergType(Types.BooleanType.get()) == AttributeType.BOOLEAN) + assert(IcebergUtil.fromIcebergType(Types.DoubleType.get()) == AttributeType.DOUBLE) + assert( + IcebergUtil.fromIcebergType(Types.TimestampType.withoutZone()) == AttributeType.TIMESTAMP + ) + assert(IcebergUtil.fromIcebergType(Types.StringType.get()) == AttributeType.STRING) + assert(IcebergUtil.fromIcebergType(Types.BinaryType.get()) == AttributeType.BINARY) + } + + it should "convert from Texera Schema to Iceberg Schema correctly" in { + assert(IcebergUtil.toIcebergSchema(texeraSchema).sameSchema(icebergSchema)) + } + + it should "convert from Iceberg Schema to Texera Schema correctly" in { + assert(IcebergUtil.fromIcebergSchema(icebergSchema) == texeraSchema) + } + + it should "convert Texera Tuple to Iceberg GenericRecord correctly" in { + val tuple = Tuple + .builder(texeraSchema) + .addSequentially( + Array( + Int.box(42), + Long.box(123456789L), + Boolean.box(true), + Double.box(3.14), + new Timestamp(10000L), + "hello world", + Array[Byte](1, 2, 3, 4) + ) + ) + .build() + + val record = IcebergUtil.toGenericRecord(tuple) + + assert(record.getField("test-1") == 42) + assert(record.getField("test-2") == 123456789L) + assert(record.getField("test-3") == true) + assert(record.getField("test-4") == 3.14) + assert(record.getField("test-5") == new Timestamp(10000L).toLocalDateTime) + assert(record.getField("test-6") == "hello world") + assert(record.getField("test-7") == ByteBuffer.wrap(Array[Byte](1, 2, 3, 4))) + + val tupleFromRecord = IcebergUtil.fromRecord(record, texeraSchema) + assert(tupleFromRecord == tuple) + } + + it should "convert Texera Tuple with null values to Iceberg GenericRecord correctly" in { + val tuple = Tuple + .builder(texeraSchema) + .addSequentially( + Array( + Int.box(42), // Non-null + null, // Null Long + Boolean.box(true), // Non-null + null, // Null Double + null, // Null Timestamp + "hello world", // Non-null String + null // Null Binary + ) + ) + .build() + + val record = IcebergUtil.toGenericRecord(tuple) + + assert(record.getField("test-1") == 42) + assert(record.getField("test-2") == null) + assert(record.getField("test-3") == true) + assert(record.getField("test-4") == null) + assert(record.getField("test-5") == null) + assert(record.getField("test-6") == "hello world") + assert(record.getField("test-7") == null) + + val tupleFromRecord = IcebergUtil.fromRecord(record, texeraSchema) + assert(tupleFromRecord == tuple) + } + + it should "convert a fully null Texera Tuple to Iceberg GenericRecord correctly" in { + val tuple = Tuple + .builder(texeraSchema) + .addSequentially( + Array( + null, // Null Integer + null, // Null Long + null, // Null Boolean + null, // Null Double + null, // Null Timestamp + null, // Null String + null // Null Binary + ) + ) + .build() + + val record = IcebergUtil.toGenericRecord(tuple) + + assert(record.getField("test-1") == null) + assert(record.getField("test-2") == null) + assert(record.getField("test-3") == null) + assert(record.getField("test-4") == null) + assert(record.getField("test-5") == null) + assert(record.getField("test-6") == null) + assert(record.getField("test-7") == null) + + val tupleFromRecord = IcebergUtil.fromRecord(record, texeraSchema) + assert(tupleFromRecord == tuple) + } + + it should "convert Iceberg GenericRecord to Texera Tuple correctly" in { + val record = GenericRecord.create(icebergSchema) + record.setField("test-1", 42) + record.setField("test-2", 123456789L) + record.setField("test-3", true) + record.setField("test-4", 3.14) + record.setField( + "test-5", + LocalDateTime.ofInstant(new Timestamp(10000L).toInstant, ZoneId.systemDefault()) + ) + record.setField("test-6", "hello world") + record.setField("test-7", ByteBuffer.wrap(Array[Byte](1, 2, 3, 4))) + + val tuple = IcebergUtil.fromRecord(record, texeraSchema) + + assert(tuple.getField[Integer]("test-1") == 42) + assert(tuple.getField[Long]("test-2") == 123456789L) + assert(tuple.getField[Boolean]("test-3") == true) + assert(tuple.getField[Double]("test-4") == 3.14) + assert(tuple.getField[Timestamp]("test-5") == new Timestamp(10000L)) + assert(tuple.getField[String]("test-6") == "hello world") + assert(tuple.getField[Array[Byte]]("test-7") sameElements Array[Byte](1, 2, 3, 4)) + } +} From 5467b6f78bd4d993d65e70a248d6402001d813e9 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 15:15:12 -0800 Subject: [PATCH 21/49] refactor the util and config --- .../src/main/resources/storage-config.yaml | 22 ++++- .../amber/core/storage/StorageConfig.scala | 96 +++++++++++++++++-- .../result/iceberg/IcebergDocument.scala | 11 +-- .../result/iceberg/IcebergTableWriter.scala | 10 +- .../edu/uci/ics/amber/util/IcebergUtil.scala | 50 ++++++---- .../result/iceberg/IcebergDocumentSpec.scala | 6 +- 6 files changed, 151 insertions(+), 44 deletions(-) diff --git a/core/workflow-core/src/main/resources/storage-config.yaml b/core/workflow-core/src/main/resources/storage-config.yaml index 2b6ed20ccaa..66db873a663 100644 --- a/core/workflow-core/src/main/resources/storage-config.yaml +++ b/core/workflow-core/src/main/resources/storage-config.yaml @@ -4,7 +4,23 @@ storage: url: "mongodb://localhost:27017" database: "texera_storage" commit-batch-size: 1000 + iceberg: + catalog: + jdbc: # currently we only support storing catalog info via jdbc, i.e. https://iceberg.apache.org/docs/1.7.1/jdbc/ + url: "jdbc:mysql://0.0.0.0:3306/texera_iceberg?serverTimezone=UTC" + username: "root" + password: "123456" + table: + commit: + batch-size: 4096 # decide the buffer size of our IcebergTableWriter + retry: + # retry configures the OCC parameter for concurrent write operations in Iceberg + # Docs about Reliability in Iceberg: https://iceberg.apache.org/docs/1.7.1/reliability/ + # Docs about full parameter list and their meaning: https://iceberg.apache.org/docs/1.7.1/configuration/#write-properties + num-retries: 10 + min-wait-ms: 100 # 0.1s + max-wait-ms: 10000 # 10s jdbc: - url: "jdbc:mysql://localhost:3306/texera_db?serverTimezone=UTC" - username: "" - password: "" \ No newline at end of file + url: "jdbc:mysql://0.0.0.0:3306/texera_db?serverTimezone=UTC" + username: "root" + password: "123456" \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index 92a676032e3..968db00f61f 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -3,6 +3,7 @@ package edu.uci.ics.amber.core.storage import edu.uci.ics.amber.util.PathUtils.corePath import org.yaml.snakeyaml.Yaml +import java.net.URI import java.util.{Map => JMap} import scala.jdk.CollectionConverters._ @@ -14,34 +15,117 @@ object StorageConfig { val storageMap = javaConf("storage").asInstanceOf[JMap[String, Any]].asScala.toMap val mongodbMap = storageMap("mongodb").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergMap = storageMap("iceberg").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergCatalogMap = icebergMap("catalog").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergCatalogJdbcMap = + icebergCatalogMap("jdbc").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergTableMap = icebergMap("table").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergCommitMap = icebergTableMap("commit").asInstanceOf[JMap[String, Any]].asScala.toMap + val icebergRetryMap = icebergCommitMap("retry").asInstanceOf[JMap[String, Any]].asScala.toMap val jdbcMap = storageMap("jdbc").asInstanceOf[JMap[String, Any]].asScala.toMap - javaConf - .updated("storage", storageMap.updated("mongodb", mongodbMap).updated("jdbc", jdbcMap)) + + javaConf.updated( + "storage", + storageMap + .updated("mongodb", mongodbMap) + .updated( + "iceberg", + icebergMap + .updated( + "catalog", + icebergCatalogMap.updated("jdbc", icebergCatalogJdbcMap) + ) + .updated( + "table", + icebergTableMap.updated( + "commit", + icebergCommitMap.updated("retry", icebergRetryMap) + ) + ) + ) + .updated("jdbc", jdbcMap) + ) } + // Result storage mode val resultStorageMode: String = conf("storage").asInstanceOf[Map[String, Any]]("result-storage-mode").asInstanceOf[String] - // For MongoDB specifics + // MongoDB configurations val mongodbUrl: String = conf("storage") .asInstanceOf[Map[String, Any]]("mongodb") .asInstanceOf[Map[String, Any]]("url") .asInstanceOf[String] + val mongodbDatabaseName: String = conf("storage") .asInstanceOf[Map[String, Any]]("mongodb") .asInstanceOf[Map[String, Any]]("database") .asInstanceOf[String] + val mongodbBatchSize: Int = conf("storage") .asInstanceOf[Map[String, Any]]("mongodb") .asInstanceOf[Map[String, Any]]("commit-batch-size") .asInstanceOf[Int] + // Iceberg catalog configurations + val icebergCatalogUrl: String = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("catalog") + .asInstanceOf[Map[String, Any]]("jdbc") + .asInstanceOf[Map[String, Any]]("url") + .asInstanceOf[String] + + val icebergCatalogUsername: String = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("catalog") + .asInstanceOf[Map[String, Any]]("jdbc") + .asInstanceOf[Map[String, Any]]("username") + .asInstanceOf[String] + + val icebergCatalogPassword: String = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("catalog") + .asInstanceOf[Map[String, Any]]("jdbc") + .asInstanceOf[Map[String, Any]]("password") + .asInstanceOf[String] + + val icebergTableCommitBatchSize: Int = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("commit") + .asInstanceOf[Map[String, Any]]("batch-size") + .asInstanceOf[Int] + + val icebergTableCommitNumRetries: Int = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("commit") + .asInstanceOf[Map[String, Any]]("retry") + .asInstanceOf[Map[String, Any]]("num-retries") + .asInstanceOf[Int] + + val icebergTableCommitMinRetryWaitMs: Int = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("commit") + .asInstanceOf[Map[String, Any]]("retry") + .asInstanceOf[Map[String, Any]]("min-wait-ms") + .asInstanceOf[Int] + + val icebergTableCommitMaxRetryWaitMs: Int = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("commit") + .asInstanceOf[Map[String, Any]]("retry") + .asInstanceOf[Map[String, Any]]("max-wait-ms") + .asInstanceOf[Int] + + // JDBC configurations val jdbcUrl: String = conf("storage") .asInstanceOf[Map[String, Any]]("jdbc") .asInstanceOf[Map[String, Any]]("url") .asInstanceOf[String] - // For jdbc specifics val jdbcUsername: String = conf("storage") .asInstanceOf[Map[String, Any]]("jdbc") .asInstanceOf[Map[String, Any]]("username") @@ -52,7 +136,7 @@ object StorageConfig { .asInstanceOf[Map[String, Any]]("password") .asInstanceOf[String] - // For file storage specifics - val fileStorageDirectoryUri = + // File storage configurations + val fileStorageDirectoryUri: URI = corePath.resolve("amber").resolve("user-resources").resolve("workflow-results").toUri } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 51705e77a62..91b946e3bdb 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -23,10 +23,9 @@ class IcebergDocument[T >: Null <: AnyRef]( ) extends VirtualDocument[T] { private val lock = new ReentrantReadWriteLock() + synchronized { - IcebergUtil - .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = true) - .get + IcebergUtil.createTable(catalog, tableNamespace, tableName, tableSchema) } /** @@ -34,7 +33,7 @@ class IcebergDocument[T >: Null <: AnyRef]( */ override def getURI: URI = { val table = IcebergUtil - .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) + .loadTable(catalog, tableNamespace, tableName) .getOrElse( throw new NoSuchTableException(f"table ${tableNamespace}.${tableName} doesn't exist") ) @@ -64,9 +63,7 @@ class IcebergDocument[T >: Null <: AnyRef]( IcebergUtil.loadTable( catalog, tableNamespace, - tableName, - tableSchema, - createIfNotExist = false + tableName ) } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 4802e0001cd..2d5c328fb9f 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -1,7 +1,8 @@ package edu.uci.ics.amber.core.storage.result.iceberg +import edu.uci.ics.amber.core.storage.StorageConfig import edu.uci.ics.amber.core.storage.model.BufferedItemWriter -import edu.uci.ics.amber.core.storage.util.StorageUtil.withLock +import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock} import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.{Schema, Table} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} @@ -24,14 +25,13 @@ class IcebergTableWriter[T]( private val lock = new ReentrantLock() private val buffer = new ArrayBuffer[T]() - override val bufferSize: Int = 3000 + override val bufferSize: Int = StorageConfig.icebergTableCommitBatchSize // Load the Iceberg table - private val table: Table = synchronized { + private val table: Table = IcebergUtil - .loadTable(catalog, tableNamespace, tableName, tableSchema, createIfNotExist = false) + .loadTable(catalog, tableNamespace, tableName) .get - } override def open(): Unit = withLock(lock) { diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 273a883416a..2dca393f833 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -8,7 +8,13 @@ import org.apache.iceberg.types.Types import org.apache.iceberg.data.{GenericRecord, Record} import org.apache.iceberg.jdbc.JdbcCatalog import org.apache.iceberg.types.Type.PrimitiveType -import org.apache.iceberg.{CatalogProperties, PartitionSpec, Table, Schema => IcebergSchema} +import org.apache.iceberg.{ + CatalogProperties, + PartitionSpec, + Table, + TableProperties, + Schema => IcebergSchema +} import java.net.URI import java.nio.ByteBuffer @@ -50,32 +56,36 @@ object IcebergUtil { catalog } - def loadTable( + def createTable( catalog: Catalog, tableNamespace: String, tableName: String, - tableSchema: IcebergSchema, - createIfNotExist: Boolean - ): Option[Table] = { + tableSchema: IcebergSchema + ): Table = { val tableProperties = Map( - "commit.retry.num-retries" -> "10", - "commit.retry.min-wait-ms" -> "10" + TableProperties.COMMIT_NUM_RETRIES -> StorageConfig.icebergTableCommitNumRetries.toString, + TableProperties.COMMIT_MAX_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMaxRetryWaitMs.toString, + TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString ) val identifier = TableIdentifier.of(tableNamespace, tableName) - if (!catalog.tableExists(identifier)) { - if (!createIfNotExist) { - return None - } - Some( - catalog.createTable( - identifier, - tableSchema, - PartitionSpec.unpartitioned, - tableProperties.asJava - ) - ) - } else { + catalog.createTable( + identifier, + tableSchema, + PartitionSpec.unpartitioned, + tableProperties.asJava + ) + } + + def loadTable( + catalog: Catalog, + tableNamespace: String, + tableName: String + ): Option[Table] = { + val identifier = TableIdentifier.of(tableNamespace, tableName) + try { Some(catalog.loadTable(identifier)) + } catch { + case exception: Exception => None } } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 88b4ee67755..ba594a1df51 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -42,9 +42,9 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { val catalog: Catalog = IcebergUtil.createJdbcCatalog( "iceberg_document_test", StorageConfig.fileStorageDirectoryUri, - StorageConfig.jdbcUrl, - StorageConfig.jdbcUsername, - StorageConfig.jdbcPassword + StorageConfig.icebergCatalogUrl, + StorageConfig.icebergCatalogUsername, + StorageConfig.icebergCatalogPassword ) val tableNamespace = "test_namespace" From d92a5ad946a3ae5db519ba43994c87646a9652d7 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 17:27:40 -0800 Subject: [PATCH 22/49] add a simple implementation for getRange and getAfter --- .../result/iceberg/IcebergDocument.scala | 8 ++++ .../storage/model/VirtualDocumentSpec.scala | 41 +++++++++++++++++++ .../result/iceberg/IcebergDocumentSpec.scala | 2 +- 3 files changed, 50 insertions(+), 1 deletion(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 91b946e3bdb..edcb3a9385c 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -121,6 +121,14 @@ class IcebergDocument[T >: Null <: AnyRef]( } } + override def getRange(from: Int, until: Int): Iterator[T] = { + get().slice(from, until) + } + + override def getAfter(offset: Int): Iterator[T] = { + get().drop(offset + 1) + } + /** * Returns a BufferedItemWriter for writing data to the table. */ diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 9b816f31ebd..35e8aedd13a 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -200,6 +200,47 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } + it should "read a specific range of items correctly" in { + val allItems = generateSampleItems() + + // Write items + val writer = document.writer() + writer.open() + allItems.foreach(writer.putOne) + writer.close() + + // Read a specific range + val from = 5 + val until = 15 + val retrievedItems = document.getRange(from, until).toList + + // Verify the retrieved range + assert( + retrievedItems.size == allItems.slice(from, until).size, + s"Items in range ($from, $until) should match." + ) + } + + it should "read items after a specific offset correctly" in { + val allItems = generateSampleItems() + + // Write items + val writer = document.writer() + writer.open() + allItems.foreach(writer.putOne) + writer.close() + + // Read items after a specific offset + val offset = 10 + val retrievedItems = document.getAfter(offset).toList + + // Verify the retrieved items + assert( + retrievedItems == allItems.drop(offset + 1), + s"Items after offset $offset should match." + ) + } + /** * Generates a sample list of items for testing. * Subclasses should override this to provide their specific sample items. diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index ba594a1df51..a7e739b083a 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -116,7 +116,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { } // Generate additional tuples with random data and occasional nulls - val additionalTuples = (1 to 20000).map { i => + val additionalTuples = (1 to 6000).map { i => Tuple .builder(amberSchema) .add( From 15508b9b6067774e3aa3823d1aabf13c4fbe239f Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 18:55:57 -0800 Subject: [PATCH 23/49] try to add iceberg as new type of result storage --- .../ics/texera/web/ComputingUnitMaster.scala | 1 + .../texera/workflow/WorkflowCompiler.scala | 4 ++- core/build.sbt | 3 +- .../src/main/resources/storage-config.yaml | 3 +- .../amber/core/storage/StorageConfig.scala | 6 ++++ .../core/storage/result/OpResultStorage.scala | 28 ++++++++++++++++++- 6 files changed, 41 insertions(+), 4 deletions(-) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala index eba2c81df24..69527bf3716 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala @@ -183,6 +183,7 @@ class ComputingUnitMaster extends io.dropwizard.Application[Configuration] with // rely on the server-side result cleanup logic. case OpResultStorage.MONGODB => MongoDatabaseManager.dropCollection(collectionName) + case OpResultStorage.ICEBERG => } }) } catch { diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala index fad33f628a0..0c94be36651 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala @@ -93,7 +93,9 @@ class WorkflowCompiler( if (!storage.contains(storageKey)) { // get the schema for result storage in certain mode val sinkStorageSchema: Option[Schema] = - if (storageType == OpResultStorage.MONGODB) { + if ( + storageType == OpResultStorage.MONGODB || storageType == OpResultStorage.ICEBERG + ) { // use the output schema on the first output port as the schema for storage Some(schema.right.get) } else { diff --git a/core/build.sbt b/core/build.sbt index bd24d6e42bf..ff37461914c 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -15,7 +15,8 @@ lazy val WorkflowCompilingService = (project in file("workflow-compiling-service .settings( dependencyOverrides ++= Seq( // override it as io.dropwizard 4 require 2.16.1 or higher - "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.16.1" + "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.16.1", + "com.fasterxml.jackson.core" % "jackson-databind" % "2.16.1", ) ) diff --git a/core/workflow-core/src/main/resources/storage-config.yaml b/core/workflow-core/src/main/resources/storage-config.yaml index 66db873a663..da7a99d2772 100644 --- a/core/workflow-core/src/main/resources/storage-config.yaml +++ b/core/workflow-core/src/main/resources/storage-config.yaml @@ -1,5 +1,5 @@ storage: - result-storage-mode: memory + result-storage-mode: iceberg mongodb: url: "mongodb://localhost:27017" database: "texera_storage" @@ -11,6 +11,7 @@ storage: username: "root" password: "123456" table: + namespace: "operator-result" commit: batch-size: 4096 # decide the buffer size of our IcebergTableWriter retry: diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index 968db00f61f..c71b94a4757 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -89,6 +89,12 @@ object StorageConfig { .asInstanceOf[Map[String, Any]]("password") .asInstanceOf[String] + val icebergTableNamespace: String = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("namespace") + .asInstanceOf[String] + val icebergTableCommitBatchSize: Int = conf("storage") .asInstanceOf[Map[String, Any]]("iceberg") .asInstanceOf[Map[String, Any]]("table") diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala index ab69d6f94d2..e2ddbd5a79b 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala @@ -3,8 +3,12 @@ package edu.uci.ics.amber.core.storage.result import com.typesafe.scalalogging.LazyLogging import edu.uci.ics.amber.core.storage.StorageConfig import edu.uci.ics.amber.core.storage.model.VirtualDocument +import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument import edu.uci.ics.amber.core.tuple.{Schema, Tuple} +import edu.uci.ics.amber.util.IcebergUtil import edu.uci.ics.amber.virtualidentity.OperatorIdentity +import org.apache.iceberg.data.Record +import org.apache.iceberg.{Schema => IcebergSchema} import java.util.concurrent.ConcurrentHashMap import scala.collection.convert.ImplicitConversions.`iterator asScala` @@ -13,6 +17,7 @@ object OpResultStorage { val defaultStorageMode: String = StorageConfig.resultStorageMode.toLowerCase val MEMORY = "memory" val MONGODB = "mongodb" + val ICEBERG = "iceberg" } /** @@ -60,7 +65,7 @@ class OpResultStorage extends Serializable with LazyLogging { val storage: VirtualDocument[Tuple] = if (mode == "memory") { new MemoryDocument[Tuple](key.id) - } else { + } else if (mode == OpResultStorage.MONGODB) { try { val fromDocument = schema.map(Tuple.fromDocument) new MongoDocument[Tuple](executionId + key, Tuple.toDocument, fromDocument) @@ -71,6 +76,27 @@ class OpResultStorage extends Serializable with LazyLogging { // fall back to memory new MemoryDocument[Tuple](key.id) } + } else { + val icebergCatalog = IcebergUtil.createJdbcCatalog( + "operator-result", + StorageConfig.fileStorageDirectoryUri, + StorageConfig.icebergCatalogUrl, + StorageConfig.icebergCatalogUsername, + StorageConfig.icebergCatalogPassword + ) + val icebergSchema = IcebergUtil.toIcebergSchema(schema.get) + val serde: Tuple => Record = tuple => IcebergUtil.toGenericRecord(tuple) + val deserde: (IcebergSchema, Record) => Tuple = (_, record) => + IcebergUtil.fromRecord(record, schema.get) + + new IcebergDocument[Tuple]( + icebergCatalog, + StorageConfig.icebergTableNamespace, + executionId + key, + icebergSchema, + serde, + deserde + ) } cache.put(key, (storage, schema)) storage From c234dfd7762f1759e5ab20786384826ab61417c5 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Fri, 20 Dec 2024 19:57:47 -0800 Subject: [PATCH 24/49] closing to fix the dependency --- core/build.sbt | 23 +++++++++++++++++++++++ core/workflow-core/build.sbt | 17 +++++++++++++---- 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/core/build.sbt b/core/build.sbt index ff37461914c..d9d98ade86b 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -17,11 +17,34 @@ lazy val WorkflowCompilingService = (project in file("workflow-compiling-service // override it as io.dropwizard 4 require 2.16.1 or higher "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.16.1", "com.fasterxml.jackson.core" % "jackson-databind" % "2.16.1", + "org.glassfish.jersey.core" % "jersey-common" % "3.0.12" ) ) lazy val WorkflowExecutionService = (project in file("amber")) .dependsOn(WorkflowOperator) + .settings( + dependencyOverrides ++= Seq( + // override it as io.dropwizard 4 require 2.16.1 or higher + "com.fasterxml.jackson.core" % "jackson-core" % "2.15.1", + "com.fasterxml.jackson.core" % "jackson-databind" % "2.15.1", + "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.15.1", + "org.slf4j" % "slf4j-api" % "1.7.26", + "org.glassfish.jersey.media" % "jersey-media-multipart" % "2.25.1", + "org.glassfish.jersey.core" % "jersey-common" % "2.25.1", + "org.glassfish.jersey.core" % "jersey-server" % "2.25.1", + "javax.xml.bind" % "jaxb-api" % "2.3.1", + "com.sun.xml.bind" % "jaxb-impl" % "2.3.1", + "org.eclipse.jetty" % "jetty-server" % "9.4.20.v20190813", + "org.eclipse.jetty" % "jetty-servlet" % "9.4.20.v20190813", + "org.eclipse.jetty" % "jetty-http" % "9.4.20.v20190813", + "org.eclipse.jetty" % "jetty-util" % "9.4.20.v20190813", + "org.eclipse.jetty" % "jetty-io" % "9.4.20.v20190813", + "org.eclipse.jetty.websocket" % "websocket-server" % "9.4.20.v20190813", + "org.eclipse.jetty.websocket" % "websocket-common" % "9.4.20.v20190813", + "org.eclipse.jetty.websocket" % "websocket-client" % "9.4.20.v20190813" + ) + ) .configs(Test) .dependsOn(DAO % "test->test") // test scope dependency diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index eb0ebad7e11..8b479d21b79 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -111,12 +111,23 @@ val arrowDependencies = Seq( libraryDependencies ++= arrowDependencies +val excludeHadoopJersey = ExclusionRule(organization = "com.sun.jersey") +val excludeHadoopSlf4j = ExclusionRule(organization = "org.slf4j") +val excludeHadoopJetty = ExclusionRule(organization = "org.eclipse.jetty") +val excludeHadoopJsp = ExclusionRule(organization = "javax.servlet.jsp") libraryDependencies ++= Seq( "org.apache.iceberg" % "iceberg-api" % "1.7.1", "org.apache.iceberg" % "iceberg-core" % "1.7.1", "org.apache.iceberg" % "iceberg-parquet" % "1.7.1", "org.apache.iceberg" % "iceberg-data" % "1.7.1", - "org.apache.hadoop" % "hadoop-client" % "3.3.1", + "org.apache.hadoop" % "hadoop-client" % "3.3.1" excludeAll( + ExclusionRule("javax.xml.bind"), + ExclusionRule("org.glassfish.jersey"), + excludeHadoopJersey, + excludeHadoopSlf4j, + excludeHadoopJetty, + excludeHadoopJsp, + ) ) ///////////////////////////////////////////////////////////////////////////// @@ -131,7 +142,5 @@ libraryDependencies ++= Seq( "com.typesafe.scala-logging" %% "scala-logging" % "3.9.5", // Scala Logging "org.eclipse.jgit" % "org.eclipse.jgit" % "5.13.0.202109080827-r", // jgit "org.yaml" % "snakeyaml" % "1.30", // yaml reader (downgrade to 1.30 due to dropwizard 1.3.23 required by amber) - "org.apache.commons" % "commons-vfs2" % "2.9.0", // for FileResolver throw VFS-related exceptions - "com.twitter" % "chill-java" % "0.10.0", // for Kryo serde/deserde - "com.twitter" %% "chill" % "0.10.0", // for Kyro serde/deserde + "org.apache.commons" % "commons-vfs2" % "2.9.0", // for FileResolver throw VFS-related exceptions // for Kyro serde/deserde ) \ No newline at end of file From 4213a5b1a4cb4e88f20d067d1d715abdaafcd5c5 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sat, 21 Dec 2024 11:25:55 -0800 Subject: [PATCH 25/49] fix the websocket connection --- core/build.sbt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/build.sbt b/core/build.sbt index d9d98ade86b..16185821874 100644 --- a/core/build.sbt +++ b/core/build.sbt @@ -38,11 +38,6 @@ lazy val WorkflowExecutionService = (project in file("amber")) "org.eclipse.jetty" % "jetty-server" % "9.4.20.v20190813", "org.eclipse.jetty" % "jetty-servlet" % "9.4.20.v20190813", "org.eclipse.jetty" % "jetty-http" % "9.4.20.v20190813", - "org.eclipse.jetty" % "jetty-util" % "9.4.20.v20190813", - "org.eclipse.jetty" % "jetty-io" % "9.4.20.v20190813", - "org.eclipse.jetty.websocket" % "websocket-server" % "9.4.20.v20190813", - "org.eclipse.jetty.websocket" % "websocket-common" % "9.4.20.v20190813", - "org.eclipse.jetty.websocket" % "websocket-client" % "9.4.20.v20190813" ) ) .configs(Test) From 11ce821d2cbfb29d7f46944076103f4f449ebe8a Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 22 Dec 2024 09:08:11 -0800 Subject: [PATCH 26/49] add create override --- .../storage/result/iceberg/IcebergDocument.scala | 14 +++++++++++++- .../scala/edu/uci/ics/amber/util/IcebergUtil.scala | 6 +++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index edcb3a9385c..e00f4379f99 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -24,8 +24,16 @@ class IcebergDocument[T >: Null <: AnyRef]( private val lock = new ReentrantReadWriteLock() + // During construction, the table gonna be created. + // If the table already exists, it will drop the existing table and create a new one synchronized { - IcebergUtil.createTable(catalog, tableNamespace, tableName, tableSchema) + IcebergUtil.createTable( + catalog, + tableNamespace, + tableName, + tableSchema, + overrideIfExists = true + ) } /** @@ -129,6 +137,10 @@ class IcebergDocument[T >: Null <: AnyRef]( get().drop(offset + 1) } + override def getCount: Long = { + get().length + } + /** * Returns a BufferedItemWriter for writing data to the table. */ diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 2dca393f833..b818f200dc5 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -60,7 +60,8 @@ object IcebergUtil { catalog: Catalog, tableNamespace: String, tableName: String, - tableSchema: IcebergSchema + tableSchema: IcebergSchema, + overrideIfExists: Boolean ): Table = { val tableProperties = Map( TableProperties.COMMIT_NUM_RETRIES -> StorageConfig.icebergTableCommitNumRetries.toString, @@ -68,6 +69,9 @@ object IcebergUtil { TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString ) val identifier = TableIdentifier.of(tableNamespace, tableName) + if (catalog.tableExists(identifier) && overrideIfExists) { + catalog.dropTable(identifier) + } catalog.createTable( identifier, tableSchema, From 129453e658b8c366f7b1360d576bc256baddfaec Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 22 Dec 2024 09:26:02 -0800 Subject: [PATCH 27/49] add more comments and adjust the dependency --- core/workflow-core/build.sbt | 2 +- .../core/storage/result/iceberg/IcebergTableWriter.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index 8b479d21b79..ea8728b3fb6 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -120,7 +120,7 @@ libraryDependencies ++= Seq( "org.apache.iceberg" % "iceberg-core" % "1.7.1", "org.apache.iceberg" % "iceberg-parquet" % "1.7.1", "org.apache.iceberg" % "iceberg-data" % "1.7.1", - "org.apache.hadoop" % "hadoop-client" % "3.3.1" excludeAll( + "org.apache.hadoop" % "hadoop-common" % "3.3.1" excludeAll( ExclusionRule("javax.xml.bind"), ExclusionRule("org.glassfish.jersey"), excludeHadoopJersey, diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 2d5c328fb9f..1d43d4a75d6 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -60,6 +60,10 @@ class IcebergTableWriter[T]( val outputFile: OutputFile = table.io().newOutputFile(filepath) // Create a Parquet data writer + // This part introduces the dependency to the Hadoop. In the source code of iceberg-parquet, see the line 160 + // https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java + // although the file is not of type HadoopOutputFile, it still creats a Hadoop Configuration() as the + // placeholder. val dataWriter: DataWriter[Record] = Parquet .writeData(outputFile) .forTable(table) @@ -68,6 +72,8 @@ class IcebergTableWriter[T]( .build() try { + // TODO: as Iceberg doesn't guarantee the order of the data written to the table, we need to think about how + // how to guarantee the order, possibly adding a additional timestamp field and use it as the sorting key buffer.foreach { item => val record = serde(item) dataWriter.write(record) From d553e3229bb252891ca3cd8643c33b91f0dddce0 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 29 Dec 2024 19:28:09 -0800 Subject: [PATCH 28/49] add worker id when creating the writer --- .../core/storage/model/VirtualDocument.scala | 3 +- .../core/storage/result/MemoryDocument.scala | 2 +- .../core/storage/result/MongoDocument.scala | 2 +- .../result/iceberg/IcebergDocument.scala | 11 ++++-- .../result/iceberg/IcebergTableWriter.scala | 9 +++-- .../storage/model/VirtualDocumentSpec.scala | 35 +++++-------------- .../operator/SpecialPhysicalOpFactory.scala | 1 + .../sink/managed/ProgressiveSinkOpExec.scala | 6 +++- 8 files changed, 34 insertions(+), 35 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/model/VirtualDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/model/VirtualDocument.scala index 19e46ce1c36..45ddf018fc9 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/model/VirtualDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/model/VirtualDocument.scala @@ -65,9 +65,10 @@ abstract class VirtualDocument[T] extends ReadonlyVirtualDocument[T] { /** * return a writer that buffers the items and performs the flush operation at close time + * @param writerIdentifier the id of the writer, maybe required by some implementations * @return a buffered item writer */ - def writer(): BufferedItemWriter[T] = + def writer(writerIdentifier: String): BufferedItemWriter[T] = throw new NotImplementedError("write method is not implemented") /** diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MemoryDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MemoryDocument.scala index c4cc26ecbcf..b508bb461c1 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MemoryDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MemoryDocument.scala @@ -69,7 +69,7 @@ class MemoryDocument[T >: Null <: AnyRef](key: String) results += item } - override def writer(): BufferedItemWriter[T] = this + override def writer(writerIdentifier: String): BufferedItemWriter[T] = this /** * The size of the buffer for the buffered writer. This number is not used currently diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MongoDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MongoDocument.scala index 18baa1844fb..f8eb42a8a91 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MongoDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/MongoDocument.scala @@ -56,7 +56,7 @@ class MongoDocument[T >: Null <: AnyRef]( * Return a buffered item writer for the MongoDB collection. * @return a new instance of MongoDBBufferedItemWriter. */ - override def writer(): BufferedItemWriter[T] = { + override def writer(writerIdentifier: String): BufferedItemWriter[T] = { new MongoDBBufferedItemWriter[T]( commitBatchSize, id, diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index e00f4379f99..c9eefcb34f4 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -144,7 +144,14 @@ class IcebergDocument[T >: Null <: AnyRef]( /** * Returns a BufferedItemWriter for writing data to the table. */ - override def writer(): BufferedItemWriter[T] = { - new IcebergTableWriter[T](catalog, tableNamespace, tableName, tableSchema, serde) + override def writer(writerIdentifier: String): BufferedItemWriter[T] = { + new IcebergTableWriter[T]( + writerIdentifier, + catalog, + tableNamespace, + tableName, + tableSchema, + serde + ) } } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 1d43d4a75d6..9aa65eed7e3 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -16,6 +16,7 @@ import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable.ArrayBuffer class IcebergTableWriter[T]( + val writerIdentifier: String, val catalog: Catalog, val tableNamespace: String, val tableName: String, @@ -25,6 +26,8 @@ class IcebergTableWriter[T]( private val lock = new ReentrantLock() private val buffer = new ArrayBuffer[T]() + // the incremental filename's index, incremented everytime a new buffer is flushed + private var filenameIdx = 0 override val bufferSize: Int = StorageConfig.icebergTableCommitBatchSize // Load the Iceberg table @@ -55,8 +58,10 @@ class IcebergTableWriter[T]( withLock(lock) { if (buffer.nonEmpty) { - // Create a unique file path using UUID - val filepath = s"${table.location()}/${UUID.randomUUID().toString}" + // Create a unique file path using writer's identifier and the filename's idx + val filepath = s"${table.location()}/${writerIdentifier}_${filenameIdx}" + // increment the idx by 1 + filenameIdx += 1 val outputFile: OutputFile = table.io().newOutputFile(filepath) // Create a Parquet data writer diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 35e8aedd13a..074041177dd 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -4,6 +4,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.time.SpanSugar.convertIntToGrainOfTime +import java.util.UUID import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.{Await, Future} @@ -38,7 +39,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { val items = generateSampleItems() // Get writer and write items - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() items.foreach(writer.putOne) writer.close() @@ -60,7 +61,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { assert(!reader.hasNext, "Reader should initially have no data.") // Write the first batch - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() batch1.foreach(writer.putOne) writer.close() @@ -70,7 +71,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { assert(retrievedBatch1.toSet == batch1.toSet, "Reader should read the first batch correctly.") // Write the second batch - val writer2 = document.writer() + val writer2 = document.writer(UUID.randomUUID().toString) writer2.open() batch2.foreach(writer2.putOne) writer2.close() @@ -83,7 +84,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { val items = generateSampleItems() // Write items - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() items.foreach(writer.putOne) writer.close() @@ -127,7 +128,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // Perform concurrent writes val writeFutures = itemBatches.map { batch => Future { - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() batch.foreach(writer.putOne) writer.close() @@ -159,7 +160,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { // Start the writer in a Future to write batches with delays val writerFuture = Future { - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() try { itemBatches.foreach { batch => @@ -204,7 +205,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { val allItems = generateSampleItems() // Write items - val writer = document.writer() + val writer = document.writer(UUID.randomUUID().toString) writer.open() allItems.foreach(writer.putOne) writer.close() @@ -221,26 +222,6 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } - it should "read items after a specific offset correctly" in { - val allItems = generateSampleItems() - - // Write items - val writer = document.writer() - writer.open() - allItems.foreach(writer.putOne) - writer.close() - - // Read items after a specific offset - val offset = 10 - val retrievedItems = document.getAfter(offset).toList - - // Verify the retrieved items - assert( - retrievedItems == allItems.drop(offset + 1), - s"Items after offset $offset should match." - ) - } - /** * Generates a sample list of items for testing. * Subclasses should override this to provide their specific sample items. diff --git a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/SpecialPhysicalOpFactory.scala b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/SpecialPhysicalOpFactory.scala index 0024993166f..67814c726cd 100644 --- a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/SpecialPhysicalOpFactory.scala +++ b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/SpecialPhysicalOpFactory.scala @@ -29,6 +29,7 @@ object SpecialPhysicalOpFactory { executionIdentity, OpExecInitInfo((idx, workers) => new ProgressiveSinkOpExec( + idx, outputMode, storageKey, workflowIdentity diff --git a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/sink/managed/ProgressiveSinkOpExec.scala b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/sink/managed/ProgressiveSinkOpExec.scala index aaeababd685..3140741a940 100644 --- a/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/sink/managed/ProgressiveSinkOpExec.scala +++ b/core/workflow-operator/src/main/scala/edu/uci/ics/amber/operator/sink/managed/ProgressiveSinkOpExec.scala @@ -10,12 +10,16 @@ import edu.uci.ics.amber.workflow.OutputPort.OutputMode import edu.uci.ics.amber.workflow.PortIdentity class ProgressiveSinkOpExec( + workerId: Int, outputMode: OutputMode, storageKey: String, workflowIdentity: WorkflowIdentity ) extends SinkOperatorExecutor { val writer: BufferedItemWriter[Tuple] = - ResultStorage.getOpResultStorage(workflowIdentity).get(OperatorIdentity(storageKey)).writer() + ResultStorage + .getOpResultStorage(workflowIdentity) + .get(OperatorIdentity(storageKey)) + .writer(workerId.toString) override def open(): Unit = { writer.open() From 78df063016d318b777e621453edc07758f3aba2a Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 29 Dec 2024 21:02:54 -0800 Subject: [PATCH 29/49] drop the write lock for iceberg table writer --- .../result/iceberg/IcebergTableWriter.scala | 94 +++++++++---------- 1 file changed, 43 insertions(+), 51 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 9aa65eed7e3..25a0041d890 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -24,7 +24,6 @@ class IcebergTableWriter[T]( val serde: T => Record ) extends BufferedItemWriter[T] { - private val lock = new ReentrantLock() private val buffer = new ArrayBuffer[T]() // the incremental filename's index, incremented everytime a new buffer is flushed private var filenameIdx = 0 @@ -37,67 +36,60 @@ class IcebergTableWriter[T]( .get override def open(): Unit = - withLock(lock) { - buffer.clear() - } + buffer.clear() - override def putOne(item: T): Unit = - withLock(lock) { - buffer.append(item) - if (buffer.size >= bufferSize) { - flushBuffer() - } + override def putOne(item: T): Unit = { + buffer.append(item) + if (buffer.size >= bufferSize) { + flushBuffer() } + } override def removeOne(item: T): Unit = - withLock(lock) { - buffer -= item - } + buffer -= item - private def flushBuffer(): Unit = - withLock(lock) { - if (buffer.nonEmpty) { + private def flushBuffer(): Unit = { + if (buffer.nonEmpty) { - // Create a unique file path using writer's identifier and the filename's idx - val filepath = s"${table.location()}/${writerIdentifier}_${filenameIdx}" - // increment the idx by 1 - filenameIdx += 1 - val outputFile: OutputFile = table.io().newOutputFile(filepath) + // Create a unique file path using writer's identifier and the filename's idx + val filepath = s"${table.location()}/${writerIdentifier}_${filenameIdx}" + // increment the idx by 1 + filenameIdx += 1 + val outputFile: OutputFile = table.io().newOutputFile(filepath) - // Create a Parquet data writer - // This part introduces the dependency to the Hadoop. In the source code of iceberg-parquet, see the line 160 - // https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java - // although the file is not of type HadoopOutputFile, it still creats a Hadoop Configuration() as the - // placeholder. - val dataWriter: DataWriter[Record] = Parquet - .writeData(outputFile) - .forTable(table) - .createWriterFunc(GenericParquetWriter.buildWriter) - .overwrite() - .build() + // Create a Parquet data writer to write a new file + // This part introduces the dependency to the Hadoop. In the source code of iceberg-parquet, see the line 160 + // https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java + // although the file is not of type HadoopOutputFile, it still creats a Hadoop Configuration() as the + // placeholder. + val dataWriter: DataWriter[Record] = Parquet + .writeData(outputFile) + .forTable(table) + .createWriterFunc(GenericParquetWriter.buildWriter) + .overwrite() + .build() - try { - // TODO: as Iceberg doesn't guarantee the order of the data written to the table, we need to think about how - // how to guarantee the order, possibly adding a additional timestamp field and use it as the sorting key - buffer.foreach { item => - val record = serde(item) - dataWriter.write(record) - } - } finally { - dataWriter.close() + try { + // TODO: as Iceberg doesn't guarantee the order of the data written to the table, we need to think about how + // how to guarantee the order, possibly adding a additional timestamp field and use it as the sorting key + buffer.foreach { item => + val record = serde(item) + dataWriter.write(record) } - - // Commit the new file to the table - val dataFile = dataWriter.toDataFile - table.newAppend().appendFile(dataFile).commit() - buffer.clear() + } finally { + dataWriter.close() } + + // Commit the new file to the table + val dataFile = dataWriter.toDataFile + table.newAppend().appendFile(dataFile).commit() + buffer.clear() } + } - override def close(): Unit = - withLock(lock) { - if (buffer.nonEmpty) { - flushBuffer() - } + override def close(): Unit = { + if (buffer.nonEmpty) { + flushBuffer() } + } } From 92e2caf06839f705d5f3063cc15f6801467b4d12 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Sun, 29 Dec 2024 21:44:31 -0800 Subject: [PATCH 30/49] clean up the build sbt --- core/workflow-core/build.sbt | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/workflow-core/build.sbt b/core/workflow-core/build.sbt index ea8728b3fb6..006994ecf50 100644 --- a/core/workflow-core/build.sbt +++ b/core/workflow-core/build.sbt @@ -111,10 +111,10 @@ val arrowDependencies = Seq( libraryDependencies ++= arrowDependencies -val excludeHadoopJersey = ExclusionRule(organization = "com.sun.jersey") -val excludeHadoopSlf4j = ExclusionRule(organization = "org.slf4j") -val excludeHadoopJetty = ExclusionRule(organization = "org.eclipse.jetty") -val excludeHadoopJsp = ExclusionRule(organization = "javax.servlet.jsp") +///////////////////////////////////////////////////////////////////////////// +// Iceberg-related Dependencies +///////////////////////////////////////////////////////////////////////////// + libraryDependencies ++= Seq( "org.apache.iceberg" % "iceberg-api" % "1.7.1", "org.apache.iceberg" % "iceberg-core" % "1.7.1", @@ -123,10 +123,10 @@ libraryDependencies ++= Seq( "org.apache.hadoop" % "hadoop-common" % "3.3.1" excludeAll( ExclusionRule("javax.xml.bind"), ExclusionRule("org.glassfish.jersey"), - excludeHadoopJersey, - excludeHadoopSlf4j, - excludeHadoopJetty, - excludeHadoopJsp, + ExclusionRule(organization = "com.sun.jersey"), + ExclusionRule(organization = "org.slf4j"), + ExclusionRule(organization = "org.eclipse.jetty"), + ExclusionRule(organization = "javax.servlet.jsp"), ) ) From bb6961ac140a996da6144a8c2b3a1b6bd5b707e7 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 08:14:41 -0800 Subject: [PATCH 31/49] fix py result storage issue --- .../texera/workflow/WorkflowCompiler.scala | 2 +- .../amber/core/storage/IcebergCatalog.scala | 28 +++++++++++++++++++ .../core/storage/result/OpResultStorage.scala | 8 ------ .../result/iceberg/IcebergDocument.scala | 4 ++- .../result/iceberg/IcebergDocumentSpec.scala | 5 ++-- 5 files changed, 35 insertions(+), 12 deletions(-) create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala index 0c94be36651..900b22458ce 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala @@ -87,7 +87,7 @@ class WorkflowCompiler( // due to the size limit of single document in mongoDB (16MB) // for sinks visualizing HTMLs which could possibly be large in size, we always use the memory storage. val storageType = { - if (outputPort.mode == SINGLE_SNAPSHOT) OpResultStorage.MEMORY + if (outputPort.mode == SINGLE_SNAPSHOT) OpResultStorage.ICEBERG else OpResultStorage.defaultStorageMode } if (!storage.contains(storageKey)) { diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala new file mode 100644 index 00000000000..274134e66c3 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala @@ -0,0 +1,28 @@ +package edu.uci.ics.amber.core.storage + +import edu.uci.ics.amber.util.IcebergUtil +import org.apache.iceberg.catalog.Catalog + +object IcebergCatalog { + private var instance: Option[Catalog] = None + + def getInstance(): Catalog = { + instance match { + case Some(catalog) => catalog + case None => + val jdbcCatalog = IcebergUtil.createJdbcCatalog( + "operator-result", + StorageConfig.fileStorageDirectoryUri, + StorageConfig.icebergCatalogUrl, + StorageConfig.icebergCatalogUsername, + StorageConfig.icebergCatalogPassword + ) + instance = Some(jdbcCatalog) + jdbcCatalog + } + } + + def replaceInstance(catalog: Catalog): Unit = { + instance = Some(catalog) + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala index e2ddbd5a79b..28b0d5dabc6 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala @@ -77,20 +77,12 @@ class OpResultStorage extends Serializable with LazyLogging { new MemoryDocument[Tuple](key.id) } } else { - val icebergCatalog = IcebergUtil.createJdbcCatalog( - "operator-result", - StorageConfig.fileStorageDirectoryUri, - StorageConfig.icebergCatalogUrl, - StorageConfig.icebergCatalogUsername, - StorageConfig.icebergCatalogPassword - ) val icebergSchema = IcebergUtil.toIcebergSchema(schema.get) val serde: Tuple => Record = tuple => IcebergUtil.toGenericRecord(tuple) val deserde: (IcebergSchema, Record) => Tuple = (_, record) => IcebergUtil.fromRecord(record, schema.get) new IcebergDocument[Tuple]( - icebergCatalog, StorageConfig.icebergTableNamespace, executionId + key, icebergSchema, diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index c9eefcb34f4..58b9a640342 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -1,5 +1,6 @@ package edu.uci.ics.amber.core.storage.result.iceberg +import edu.uci.ics.amber.core.storage.IcebergCatalog import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock, withWriteLock} import edu.uci.ics.amber.util.IcebergUtil @@ -14,7 +15,6 @@ import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock} import scala.jdk.CollectionConverters._ class IcebergDocument[T >: Null <: AnyRef]( - val catalog: Catalog, val tableNamespace: String, val tableName: String, val tableSchema: org.apache.iceberg.Schema, @@ -24,6 +24,8 @@ class IcebergDocument[T >: Null <: AnyRef]( private val lock = new ReentrantReadWriteLock() + @transient lazy val catalog: Catalog = IcebergCatalog.getInstance() + // During construction, the table gonna be created. // If the table already exists, it will drop the existing table and create a new one synchronized { diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index a7e739b083a..a81267a5f0e 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.storage.result.iceberg -import edu.uci.ics.amber.core.storage.StorageConfig +import edu.uci.ics.amber.core.storage.{IcebergCatalog, StorageConfig} import edu.uci.ics.amber.core.storage.model.VirtualDocumentSpec import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} @@ -47,6 +47,8 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { StorageConfig.icebergCatalogPassword ) + IcebergCatalog.replaceInstance(catalog) + val tableNamespace = "test_namespace" var tableName: String = _ @@ -59,7 +61,6 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { // Implementation of getDocument override def getDocument: IcebergDocument[Tuple] = { new IcebergDocument[Tuple]( - catalog, tableNamespace, tableName, icebergSchema, From 1be10bf7b2ab19a4200576e86fd10e396658f999 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 09:39:03 -0800 Subject: [PATCH 32/49] clean up the iceberg document --- .../result/iceberg/IcebergDocument.scala | 81 ++++++++++++++----- .../result/iceberg/IcebergTableWriter.scala | 2 +- .../edu/uci/ics/amber/util/IcebergUtil.scala | 2 +- 3 files changed, 61 insertions(+), 24 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 58b9a640342..ac87f461a6e 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -14,6 +14,20 @@ import java.net.URI import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock} import scala.jdk.CollectionConverters._ +/** + * IcebergDocument is used to read and write a set of T as an Iceberg table. + * It provides iterator-based read methods and supports multiple writers to write to the same table. + * + * - On construction, the table will be created if it does not exist. + * - If the table exists, it will be overridden. + * + * @param tableNamespace namespace of the table. + * @param tableName name of the table. + * @param tableSchema schema of the table. + * @param serde function to serialize T into an Iceberg Record. + * @param deserde function to deserialize an Iceberg Record into T. + * @tparam T type of the data items stored in the Iceberg table. + */ class IcebergDocument[T >: Null <: AnyRef]( val tableNamespace: String, val tableName: String, @@ -26,8 +40,7 @@ class IcebergDocument[T >: Null <: AnyRef]( @transient lazy val catalog: Catalog = IcebergCatalog.getInstance() - // During construction, the table gonna be created. - // If the table already exists, it will drop the existing table and create a new one + // During construction, create or override the table synchronized { IcebergUtil.createTable( catalog, @@ -40,10 +53,11 @@ class IcebergDocument[T >: Null <: AnyRef]( /** * Returns the URI of the table location. + * @throws NoSuchTableException if the table does not exist. */ override def getURI: URI = { val table = IcebergUtil - .loadTable(catalog, tableNamespace, tableName) + .loadTableMetadata(catalog, tableNamespace, tableName) .getOrElse( throw new NoSuchTableException(f"table ${tableNamespace}.${tableName} doesn't exist") ) @@ -51,7 +65,7 @@ class IcebergDocument[T >: Null <: AnyRef]( } /** - * Deletes the table. + * Deletes the table and clears its contents. */ override def clear(): Unit = withWriteLock(lock) { @@ -61,47 +75,60 @@ class IcebergDocument[T >: Null <: AnyRef]( } } + /** + * Get an iterator for reading records from the table. + */ override def get(): Iterator[T] = withReadLock(lock) { new Iterator[T] { private val iteLock = new ReentrantLock() - private var table: Option[Table] = loadTable() + // Load the table instance, initially the table instance may not exists + private var table: Option[Table] = loadTableMetadata() + + // Last seen snapshot id(logically it's like a version number). While reading, new snapshots may be created private var lastSnapshotId: Option[Long] = None + + // Iterator for the records private var recordIterator: Iterator[T] = loadRecords() - private def loadTable(): Option[Table] = { - IcebergUtil.loadTable( + // Util function to load the table's metadata + private def loadTableMetadata(): Option[Table] = { + IcebergUtil.loadTableMetadata( catalog, tableNamespace, tableName ) } - /** - * Loads records incrementally using `newIncrementalAppendScan` from the last snapshot ID. - */ + // Util function to load new records when current iterator reach to EOF private def loadRecords(): Iterator[T] = withLock(iteLock) { table match { case Some(t) => - val currentSnapshot = Option(t.currentSnapshot()) - val currentSnapshotId = currentSnapshot.map(_.snapshotId()) + val currentSnapshotId = Option(t.currentSnapshot()).map(_.snapshotId()) val records: CloseableIterable[Record] = (lastSnapshotId, currentSnapshotId) match { + // case1: the read hasn't started yet(because the lastSnapshotId is None) + // - create a iterator that will read from the beginning of the table + case (None, Some(_)) => + IcebergGenerics.read(t).build() + + // case2: the read is ongoing and two Ids are not equal case (Some(lastId), Some(currId)) if lastId != currId => - // Perform incremental append scan if snapshot IDs are different + // This means that the new snapshots have been produced since last read, thus + // create a iterator that only reads the new data IcebergGenerics.read(t).appendsAfter(lastId).build() - case (None, Some(_)) => - // First read, perform a full scan - IcebergGenerics.read(t).build() + // case3: the read is ongoing and two Ids are equal + case (Some(lastId), Some(currId)) if lastId == currId => + // This means that there is no new data during the read, thus no new record + CloseableIterable.empty() + // default case: Both Ids are None, meaning no data yet. case _ => - // No new data; return an empty iterator CloseableIterable.empty() } - // Update the last snapshot ID to the current one lastSnapshotId = currentSnapshotId records.iterator().asScala.map(record => deserde(tableSchema, record)) @@ -113,14 +140,13 @@ class IcebergDocument[T >: Null <: AnyRef]( if (recordIterator.hasNext) { true } else { - // Refresh the table and check for new commits + // Refresh table and check for new data if (table.isEmpty) { - table = loadTable() + table = loadTableMetadata() } table.foreach(_.refresh()) recordIterator = loadRecords() recordIterator.hasNext - } } @@ -131,20 +157,31 @@ class IcebergDocument[T >: Null <: AnyRef]( } } + /** + * Get records within a specified range [from, until). + */ override def getRange(from: Int, until: Int): Iterator[T] = { get().slice(from, until) } + /** + * Get records starting after a specified offset. + */ override def getAfter(offset: Int): Iterator[T] = { get().drop(offset + 1) } + /** + * Get the total count of records in the table. + */ override def getCount: Long = { get().length } /** - * Returns a BufferedItemWriter for writing data to the table. + * Creates a BufferedItemWriter for writing data to the table. + * @param writerIdentifier The writer's ID. It should be unique within the same table, as each writer will use it as + * the prefix of the files they append */ override def writer(writerIdentifier: String): BufferedItemWriter[T] = { new IcebergTableWriter[T]( diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index 25a0041d890..d050da3f32b 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -32,7 +32,7 @@ class IcebergTableWriter[T]( // Load the Iceberg table private val table: Table = IcebergUtil - .loadTable(catalog, tableNamespace, tableName) + .loadTableMetadata(catalog, tableNamespace, tableName) .get override def open(): Unit = diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index b818f200dc5..3b557338abd 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -80,7 +80,7 @@ object IcebergUtil { ) } - def loadTable( + def loadTableMetadata( catalog: Catalog, tableNamespace: String, tableName: String From 7adfda4a69656d97031adff5d000d8273c5a6d17 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 09:51:44 -0800 Subject: [PATCH 33/49] clean up the iceberg writer --- .../result/iceberg/IcebergTableWriter.scala | 53 +++++++++++++++---- 1 file changed, 43 insertions(+), 10 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index d050da3f32b..e945968b791 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -2,19 +2,31 @@ package edu.uci.ics.amber.core.storage.result.iceberg import edu.uci.ics.amber.core.storage.StorageConfig import edu.uci.ics.amber.core.storage.model.BufferedItemWriter -import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock} import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.{Schema, Table} -import org.apache.iceberg.catalog.{Catalog, TableIdentifier} +import org.apache.iceberg.catalog.Catalog import org.apache.iceberg.data.Record import org.apache.iceberg.data.parquet.GenericParquetWriter import org.apache.iceberg.io.{DataWriter, OutputFile} import org.apache.iceberg.parquet.Parquet - -import java.util.UUID -import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable.ArrayBuffer +/** + * IcebergTableWriter writes data to the given Iceberg table in an append-only way. + * - Each time the buffer is flushed, a new data file is created with a unique name. + * - The `writerIdentifier` is used to prefix the created files. + * - Iceberg data files are immutable once created. So each flush will create a distinct file. + * + * **Thread Safety**: This writer is **NOT thread-safe**, so only one thread should call this writer. + * + * @param writerIdentifier a unique identifier used to prefix the created files. + * @param catalog the Iceberg catalog to manage table metadata. + * @param tableNamespace the namespace of the Iceberg table. + * @param tableName the name of the Iceberg table. + * @param tableSchema the schema of the Iceberg table. + * @param serde a function to serialize `T` into an Iceberg `Record`. + * @tparam T the type of the data items written to the table. + */ class IcebergTableWriter[T]( val writerIdentifier: String, val catalog: Catalog, @@ -24,8 +36,9 @@ class IcebergTableWriter[T]( val serde: T => Record ) extends BufferedItemWriter[T] { + // Buffer to hold items before flushing to the table private val buffer = new ArrayBuffer[T]() - // the incremental filename's index, incremented everytime a new buffer is flushed + // Incremental filename index, incremented each time a new buffer is flushed private var filenameIdx = 0 override val bufferSize: Int = StorageConfig.icebergTableCommitBatchSize @@ -35,9 +48,17 @@ class IcebergTableWriter[T]( .loadTableMetadata(catalog, tableNamespace, tableName) .get + /** + * Open the writer and clear the buffer. + */ override def open(): Unit = buffer.clear() + /** + * Add a single item to the buffer. + * - If the buffer size exceeds the configured limit, the buffer is flushed. + * @param item the item to add to the buffer. + */ override def putOne(item: T): Unit = { buffer.append(item) if (buffer.size >= bufferSize) { @@ -45,15 +66,24 @@ class IcebergTableWriter[T]( } } + /** + * Remove a single item from the buffer. + * @param item the item to remove from the buffer. + */ override def removeOne(item: T): Unit = buffer -= item + /** + * Flush the current buffer to a new Iceberg data file. + * - Creates a new data file using the writer identifier and an incremental filename index. + * - Writes all buffered items to the new file and commits it to the Iceberg table. + */ private def flushBuffer(): Unit = { if (buffer.nonEmpty) { - // Create a unique file path using writer's identifier and the filename's idx + // Create a unique file path using the writer's identifier and the filename index val filepath = s"${table.location()}/${writerIdentifier}_${filenameIdx}" - // increment the idx by 1 + // Increment the filename index by 1 filenameIdx += 1 val outputFile: OutputFile = table.io().newOutputFile(filepath) @@ -69,9 +99,8 @@ class IcebergTableWriter[T]( .overwrite() .build() + // Write each buffered item to the data file try { - // TODO: as Iceberg doesn't guarantee the order of the data written to the table, we need to think about how - // how to guarantee the order, possibly adding a additional timestamp field and use it as the sorting key buffer.foreach { item => val record = serde(item) dataWriter.write(record) @@ -83,10 +112,14 @@ class IcebergTableWriter[T]( // Commit the new file to the table val dataFile = dataWriter.toDataFile table.newAppend().appendFile(dataFile).commit() + buffer.clear() } } + /** + * Close the writer, ensuring any remaining buffered items are flushed. + */ override def close(): Unit = { if (buffer.nonEmpty) { flushBuffer() From 4617564f6c36c10a8de87bb2b76a155263772257 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 09:58:27 -0800 Subject: [PATCH 34/49] add more comments on the iceberg util --- .../edu/uci/ics/amber/util/IcebergUtil.scala | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 3b557338abd..6324415739f 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -23,6 +23,9 @@ import java.time.LocalDateTime import java.time.ZoneId import scala.jdk.CollectionConverters._ +/** + * Util functions to interact with Iceberg Tables + */ object IcebergUtil { /** @@ -56,6 +59,18 @@ object IcebergUtil { catalog } + /** + * Creates a new Iceberg table with the specified schema and properties. + * - Drops the existing table if `overrideIfExists` is true and the table already exists. + * - Creates an unpartitioned table with custom commit retry properties. + * + * @param catalog the Iceberg catalog to manage the table. + * @param tableNamespace the namespace of the table. + * @param tableName the name of the table. + * @param tableSchema the schema of the table. + * @param overrideIfExists whether to drop and recreate the table if it exists. + * @return the created Iceberg table. + */ def createTable( catalog: Catalog, tableNamespace: String, @@ -80,6 +95,16 @@ object IcebergUtil { ) } + /** + * Loads metadata for an existing Iceberg table. + * - Returns `Some(Table)` if the table exists and is successfully loaded. + * - Returns `None` if the table does not exist or cannot be loaded. + * + * @param catalog the Iceberg catalog to load the table from. + * @param tableNamespace the namespace of the table. + * @param tableName the name of the table. + * @return an Option containing the table, or None if not found. + */ def loadTableMetadata( catalog: Catalog, tableNamespace: String, @@ -89,7 +114,7 @@ object IcebergUtil { try { Some(catalog.loadTable(identifier)) } catch { - case exception: Exception => None + case _: Exception => None } } @@ -152,7 +177,7 @@ object IcebergUtil { } /** - * Converts an Iceberg `Record` to an Amber `Tuple`, handling `null` values. + * Converts an Iceberg `Record` to an Amber `Tuple` * * @param record The Iceberg Record. * @param amberSchema The corresponding Amber Schema. @@ -176,7 +201,7 @@ object IcebergUtil { } /** - * Converts an Iceberg `Schema` to a custom Amber `Schema`. + * Converts an Iceberg `Schema` to an Amber `Schema`. * * @param icebergSchema The Iceberg Schema. * @return The corresponding Amber Schema. @@ -194,7 +219,7 @@ object IcebergUtil { } /** - * Converts an Iceberg `Type` to a custom Amber `AttributeType`. + * Converts an Iceberg `Type` to an Amber `AttributeType`. * * @param icebergType The Iceberg Type. * @return The corresponding Amber AttributeType. From 13731cb64491cfdae3906866ac13b71760884c5d Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 10:16:55 -0800 Subject: [PATCH 35/49] add more comments --- .../amber/core/storage/IcebergCatalog.scala | 28 -------- .../core/storage/IcebergCatalogInstance.scala | 46 +++++++++++++ .../result/iceberg/IcebergDocument.scala | 4 +- .../result/iceberg/fileio/LocalFileIO.scala | 44 +++++++++++- .../iceberg/fileio/LocalInputFile.scala | 67 ++++++++++++++++++- .../iceberg/fileio/LocalOutputFile.scala | 56 +++++++++++++++- .../edu/uci/ics/amber/util/IcebergUtil.scala | 4 +- .../result/iceberg/IcebergDocumentSpec.scala | 4 +- 8 files changed, 215 insertions(+), 38 deletions(-) delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala create mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalogInstance.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala deleted file mode 100644 index 274134e66c3..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalog.scala +++ /dev/null @@ -1,28 +0,0 @@ -package edu.uci.ics.amber.core.storage - -import edu.uci.ics.amber.util.IcebergUtil -import org.apache.iceberg.catalog.Catalog - -object IcebergCatalog { - private var instance: Option[Catalog] = None - - def getInstance(): Catalog = { - instance match { - case Some(catalog) => catalog - case None => - val jdbcCatalog = IcebergUtil.createJdbcCatalog( - "operator-result", - StorageConfig.fileStorageDirectoryUri, - StorageConfig.icebergCatalogUrl, - StorageConfig.icebergCatalogUsername, - StorageConfig.icebergCatalogPassword - ) - instance = Some(jdbcCatalog) - jdbcCatalog - } - } - - def replaceInstance(catalog: Catalog): Unit = { - instance = Some(catalog) - } -} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalogInstance.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalogInstance.scala new file mode 100644 index 00000000000..a1288f14264 --- /dev/null +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/IcebergCatalogInstance.scala @@ -0,0 +1,46 @@ +package edu.uci.ics.amber.core.storage + +import edu.uci.ics.amber.util.IcebergUtil +import org.apache.iceberg.catalog.Catalog + +/** + * IcebergCatalogInstance is a singleton that manages the Iceberg catalog instance. + * - Provides a single shared catalog for all Iceberg table-related operations in the Texera application. + * - Lazily initializes the catalog on first access. + * - Supports replacing the catalog instance primarily for testing or reconfiguration. + */ +object IcebergCatalogInstance { + + private var instance: Option[Catalog] = None + + /** + * Retrieves the singleton Iceberg catalog instance. + * - If the catalog is not initialized, it is lazily created using the configured properties. + * @return the Iceberg catalog instance. + */ + def getInstance(): Catalog = { + instance match { + case Some(catalog) => catalog + case None => + val jdbcCatalog = IcebergUtil.createJdbcCatalog( + "texera-iceberg", + StorageConfig.fileStorageDirectoryUri, + StorageConfig.icebergCatalogUrl, + StorageConfig.icebergCatalogUsername, + StorageConfig.icebergCatalogPassword + ) + instance = Some(jdbcCatalog) + jdbcCatalog + } + } + + /** + * Replaces the existing Iceberg catalog instance. + * - This method is useful for testing or dynamically updating the catalog. + * + * @param catalog the new Iceberg catalog instance to replace the current one. + */ + def replaceInstance(catalog: Catalog): Unit = { + instance = Some(catalog) + } +} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index ac87f461a6e..395c0cb1bee 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.core.storage.result.iceberg -import edu.uci.ics.amber.core.storage.IcebergCatalog +import edu.uci.ics.amber.core.storage.IcebergCatalogInstance import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock, withWriteLock} import edu.uci.ics.amber.util.IcebergUtil @@ -38,7 +38,7 @@ class IcebergDocument[T >: Null <: AnyRef]( private val lock = new ReentrantReadWriteLock() - @transient lazy val catalog: Catalog = IcebergCatalog.getInstance() + @transient lazy val catalog: Catalog = IcebergCatalogInstance.getInstance() // During construction, create or override the table synchronized { diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala index 19dd6af7102..4df91da7a7f 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala @@ -3,24 +3,50 @@ package edu.uci.ics.amber.core.storage.result.iceberg.fileio import org.apache.iceberg.io.{FileIO, InputFile, OutputFile} import org.apache.iceberg.exceptions.RuntimeIOException -import java.io.{File, IOException, RandomAccessFile} -import java.nio.file.{Files, Paths, StandardOpenOption} +import java.io.IOException +import java.nio.file.{Files, Paths} import java.util.concurrent.atomic.AtomicBoolean +/** + * LocalFileIO is a custom FileIO implementation for Iceberg to use the local file system + * for storing data and metadata files. + */ class LocalFileIO extends FileIO { + // Tracks whether the FileIO instance is closed private val isClosed = new AtomicBoolean(false) + /** + * Creates an InputFile for reading from the local file system. + * + * @param path the file path. + * @return a new LocalInputFile instance. + * @throws IllegalStateException if the FileIO is closed. + */ override def newInputFile(path: String): InputFile = { ensureNotClosed() new LocalInputFile(path) } + /** + * Creates an OutputFile for writing to the local file system. + * + * @param path the file path. + * @return a new LocalOutputFile instance. + * @throws IllegalStateException if the FileIO is closed. + */ override def newOutputFile(path: String): OutputFile = { ensureNotClosed() new LocalOutputFile(path) } + /** + * Deletes a file from the local file system. + * + * @param path the file path to delete. + * @throws RuntimeIOException if the deletion fails. + * @throws IllegalStateException if the FileIO is closed. + */ override def deleteFile(path: String): Unit = { ensureNotClosed() try { @@ -30,14 +56,28 @@ class LocalFileIO extends FileIO { } } + /** + * Initializes the FileIO with properties. + * - No special initialization is required for local file systems. + * + * @param properties configuration properties (unused). + */ override def initialize(properties: java.util.Map[String, String]): Unit = { // No special initialization required for local file system } + /** + * Marks the FileIO as closed, preventing further operations. + */ override def close(): Unit = { isClosed.set(true) } + /** + * Ensures the FileIO is not closed before performing operations. + * + * @throws IllegalStateException if the FileIO is closed. + */ private def ensureNotClosed(): Unit = { if (isClosed.get()) { throw new IllegalStateException("Cannot use LocalFileIO after it has been closed") diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala index 9301e523d9c..efe1a2f8481 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala @@ -3,13 +3,28 @@ package edu.uci.ics.amber.core.storage.result.iceberg.fileio import org.apache.iceberg.io.{InputFile, SeekableInputStream} import org.apache.iceberg.exceptions.RuntimeIOException -import java.io.{File, FileInputStream, IOException, RandomAccessFile} +import java.io.{IOException, RandomAccessFile} import java.nio.file.{Files, Paths} +/** + * LocalInputFile is an Iceberg InputFile implementation for reading files from the local file system.* + * @param path the local file path. + */ class LocalInputFile(path: String) extends InputFile { + /** + * Returns the file's location as a string (its path). + * + * @return the file location. + */ override def location(): String = path + /** + * Gets the length of the file in bytes. + * + * @return the file length. + * @throws RuntimeIOException if the file length cannot be determined. + */ override def getLength: Long = { try { Files.size(Paths.get(path)) @@ -18,6 +33,12 @@ class LocalInputFile(path: String) extends InputFile { } } + /** + * Opens a new seekable input stream for the file. + * + * @return a new SeekableFileInputStream instance. + * @throws RuntimeIOException if the file cannot be opened. + */ override def newStream(): SeekableInputStream = { try { new SeekableFileInputStream(path) @@ -27,23 +48,67 @@ class LocalInputFile(path: String) extends InputFile { } } + /** + * Checks whether the file exists on the local file system. + * + * @return true if the file exists, false otherwise. + */ override def exists(): Boolean = Files.exists(Paths.get(path)) } +/** + * SeekableFileInputStream is a seekable input stream for reading files from the local file system. + * - Implements Iceberg's SeekableInputStream for efficient random access. + * + * @param path the local file path. + */ class SeekableFileInputStream(path: String) extends SeekableInputStream { + // Underlying RandomAccessFile for low-level file operations private val file = new RandomAccessFile(path, "r") + /** + * Reads the next byte from the file. + * + * @return the next byte, or -1 if the end of the file is reached. + */ override def read(): Int = file.read() + /** + * Reads a sequence of bytes into the given array. + * + * @param b the byte array to store data. + * @param off the start offset in the array. + * @param len the number of bytes to read. + * @return the number of bytes read, or -1 if the end of the file is reached. + */ override def read(b: Array[Byte], off: Int, len: Int): Int = file.read(b, off, len) + /** + * Seeks to the specified position in the file. + * + * @param pos the position to seek to. + */ override def seek(pos: Long): Unit = file.seek(pos) + /** + * Gets the current position in the file. + * + * @return the current file pointer position. + */ override def getPos: Long = file.getFilePointer + /** + * Closes the input stream and releases any system resources. + */ override def close(): Unit = file.close() + /** + * Skips the specified number of bytes in the file. + * + * @param n the number of bytes to skip. + * @return the actual number of bytes skipped. + */ override def skip(n: Long): Long = { val currentPos = file.getFilePointer val fileLength = file.length() diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala index fa959349d76..4e2844e8ffc 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala @@ -4,12 +4,27 @@ import org.apache.iceberg.io.{OutputFile, PositionOutputStream} import org.apache.iceberg.exceptions.RuntimeIOException import java.io.{File, FileOutputStream, IOException} -import java.nio.file.{Files, Paths, StandardOpenOption} +/** + * LocalOutputFile is an Iceberg OutputFile implementation for writing files to the local file system. + * + * @param path the local file path. + */ class LocalOutputFile(path: String) extends OutputFile { + /** + * Returns the file's location as a string (its path). + * + * @return the file location. + */ override def location(): String = path + /** + * Creates a new file for writing. + * + * @return a new LocalPositionOutputStream instance. + * @throws RuntimeIOException if the file cannot be created. + */ override def create(): PositionOutputStream = { try { new LocalPositionOutputStream(path, append = false) @@ -18,6 +33,12 @@ class LocalOutputFile(path: String) extends OutputFile { } } + /** + * Creates or overwrites a file for writing. + * + * @return a new LocalPositionOutputStream instance. + * @throws RuntimeIOException if the file cannot be created or overwritten. + */ override def createOrOverwrite(): PositionOutputStream = { try { new LocalPositionOutputStream(path, append = false) @@ -27,9 +48,22 @@ class LocalOutputFile(path: String) extends OutputFile { } } + /** + * Converts this OutputFile to a LocalInputFile for reading. + * + * @return a LocalInputFile instance. + */ override def toInputFile: LocalInputFile = new LocalInputFile(path) } +/** + * LocalPositionOutputStream is a PositionOutputStream implementation for writing to local files. + * - Tracks the write position to support Iceberg's file IO abstractions. + * - Ensures parent directories are created before writing. + * + * @param path the local file path. + * @param append whether to append to the file (if it exists). + */ class LocalPositionOutputStream(path: String, append: Boolean) extends PositionOutputStream { // Ensure the parent directories exist @@ -41,17 +75,37 @@ class LocalPositionOutputStream(path: String, append: Boolean) extends PositionO private val outputStream = new FileOutputStream(file, append) private var position: Long = 0 + /** + * Writes a single byte to the file and updates the position. + * + * @param b the byte to write. + */ override def write(b: Int): Unit = { outputStream.write(b) position += 1 } + /** + * Writes a sequence of bytes to the file and updates the position. + * + * @param b the byte array containing data. + * @param off the start offset in the array. + * @param len the number of bytes to write. + */ override def write(b: Array[Byte], off: Int, len: Int): Unit = { outputStream.write(b, off, len) position += len } + /** + * Gets the current write position in the file. + * + * @return the current position in bytes. + */ override def getPos: Long = position + /** + * Closes the output stream and releases any system resources. + */ override def close(): Unit = outputStream.close() } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 6324415739f..52e7a681241 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -24,8 +24,8 @@ import java.time.ZoneId import scala.jdk.CollectionConverters._ /** - * Util functions to interact with Iceberg Tables - */ + * Util functions to interact with Iceberg Tables + */ object IcebergUtil { /** diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index a81267a5f0e..0e31c51a7c8 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -1,6 +1,6 @@ package edu.uci.ics.amber.storage.result.iceberg -import edu.uci.ics.amber.core.storage.{IcebergCatalog, StorageConfig} +import edu.uci.ics.amber.core.storage.{IcebergCatalogInstance, StorageConfig} import edu.uci.ics.amber.core.storage.model.VirtualDocumentSpec import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} @@ -47,7 +47,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { StorageConfig.icebergCatalogPassword ) - IcebergCatalog.replaceInstance(catalog) + IcebergCatalogInstance.replaceInstance(catalog) val tableNamespace = "test_namespace" var tableName: String = _ From 2baa66189c21132063256c4a9cc8c253bb4b5839 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 10:25:12 -0800 Subject: [PATCH 36/49] refactor local file IO --- .../iceberg/{fileio => }/LocalFileIO.scala | 9 +- .../iceberg/fileio/LocalInputFile.scala | 119 ------------------ .../iceberg/fileio/LocalOutputFile.scala | 111 ---------------- .../edu/uci/ics/amber/util/IcebergUtil.scala | 2 +- 4 files changed, 6 insertions(+), 235 deletions(-) rename core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/{fileio => }/LocalFileIO.scala (93%) delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala delete mode 100644 core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/LocalFileIO.scala similarity index 93% rename from core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala rename to core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/LocalFileIO.scala index 4df91da7a7f..1cd1b873fc6 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalFileIO.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/LocalFileIO.scala @@ -1,7 +1,8 @@ -package edu.uci.ics.amber.core.storage.result.iceberg.fileio +package edu.uci.ics.amber.core.storage.result.iceberg -import org.apache.iceberg.io.{FileIO, InputFile, OutputFile} +import org.apache.iceberg.Files.{localInput, localOutput} import org.apache.iceberg.exceptions.RuntimeIOException +import org.apache.iceberg.io.{FileIO, InputFile, OutputFile} import java.io.IOException import java.nio.file.{Files, Paths} @@ -25,7 +26,7 @@ class LocalFileIO extends FileIO { */ override def newInputFile(path: String): InputFile = { ensureNotClosed() - new LocalInputFile(path) + localInput(path) } /** @@ -37,7 +38,7 @@ class LocalFileIO extends FileIO { */ override def newOutputFile(path: String): OutputFile = { ensureNotClosed() - new LocalOutputFile(path) + localOutput(path) } /** diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala deleted file mode 100644 index efe1a2f8481..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalInputFile.scala +++ /dev/null @@ -1,119 +0,0 @@ -package edu.uci.ics.amber.core.storage.result.iceberg.fileio - -import org.apache.iceberg.io.{InputFile, SeekableInputStream} -import org.apache.iceberg.exceptions.RuntimeIOException - -import java.io.{IOException, RandomAccessFile} -import java.nio.file.{Files, Paths} - -/** - * LocalInputFile is an Iceberg InputFile implementation for reading files from the local file system.* - * @param path the local file path. - */ -class LocalInputFile(path: String) extends InputFile { - - /** - * Returns the file's location as a string (its path). - * - * @return the file location. - */ - override def location(): String = path - - /** - * Gets the length of the file in bytes. - * - * @return the file length. - * @throws RuntimeIOException if the file length cannot be determined. - */ - override def getLength: Long = { - try { - Files.size(Paths.get(path)) - } catch { - case e: IOException => throw new RuntimeIOException(e, s"Failed to get length of file: $path") - } - } - - /** - * Opens a new seekable input stream for the file. - * - * @return a new SeekableFileInputStream instance. - * @throws RuntimeIOException if the file cannot be opened. - */ - override def newStream(): SeekableInputStream = { - try { - new SeekableFileInputStream(path) - } catch { - case e: IOException => - throw new RuntimeIOException(e, s"Failed to open file for reading: $path") - } - } - - /** - * Checks whether the file exists on the local file system. - * - * @return true if the file exists, false otherwise. - */ - override def exists(): Boolean = Files.exists(Paths.get(path)) -} - -/** - * SeekableFileInputStream is a seekable input stream for reading files from the local file system. - * - Implements Iceberg's SeekableInputStream for efficient random access. - * - * @param path the local file path. - */ -class SeekableFileInputStream(path: String) extends SeekableInputStream { - - // Underlying RandomAccessFile for low-level file operations - private val file = new RandomAccessFile(path, "r") - - /** - * Reads the next byte from the file. - * - * @return the next byte, or -1 if the end of the file is reached. - */ - override def read(): Int = file.read() - - /** - * Reads a sequence of bytes into the given array. - * - * @param b the byte array to store data. - * @param off the start offset in the array. - * @param len the number of bytes to read. - * @return the number of bytes read, or -1 if the end of the file is reached. - */ - override def read(b: Array[Byte], off: Int, len: Int): Int = file.read(b, off, len) - - /** - * Seeks to the specified position in the file. - * - * @param pos the position to seek to. - */ - override def seek(pos: Long): Unit = file.seek(pos) - - /** - * Gets the current position in the file. - * - * @return the current file pointer position. - */ - override def getPos: Long = file.getFilePointer - - /** - * Closes the input stream and releases any system resources. - */ - override def close(): Unit = file.close() - - /** - * Skips the specified number of bytes in the file. - * - * @param n the number of bytes to skip. - * @return the actual number of bytes skipped. - */ - override def skip(n: Long): Long = { - val currentPos = file.getFilePointer - val fileLength = file.length() - val newPos = Math.min(currentPos + n, fileLength) - file.seek(newPos) - newPos - currentPos - } -} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala deleted file mode 100644 index 4e2844e8ffc..00000000000 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/fileio/LocalOutputFile.scala +++ /dev/null @@ -1,111 +0,0 @@ -package edu.uci.ics.amber.core.storage.result.iceberg.fileio - -import org.apache.iceberg.io.{OutputFile, PositionOutputStream} -import org.apache.iceberg.exceptions.RuntimeIOException - -import java.io.{File, FileOutputStream, IOException} - -/** - * LocalOutputFile is an Iceberg OutputFile implementation for writing files to the local file system. - * - * @param path the local file path. - */ -class LocalOutputFile(path: String) extends OutputFile { - - /** - * Returns the file's location as a string (its path). - * - * @return the file location. - */ - override def location(): String = path - - /** - * Creates a new file for writing. - * - * @return a new LocalPositionOutputStream instance. - * @throws RuntimeIOException if the file cannot be created. - */ - override def create(): PositionOutputStream = { - try { - new LocalPositionOutputStream(path, append = false) - } catch { - case e: IOException => throw new RuntimeIOException(e, s"Failed to create file: $path") - } - } - - /** - * Creates or overwrites a file for writing. - * - * @return a new LocalPositionOutputStream instance. - * @throws RuntimeIOException if the file cannot be created or overwritten. - */ - override def createOrOverwrite(): PositionOutputStream = { - try { - new LocalPositionOutputStream(path, append = false) - } catch { - case e: IOException => - throw new RuntimeIOException(e, s"Failed to create or overwrite file: $path") - } - } - - /** - * Converts this OutputFile to a LocalInputFile for reading. - * - * @return a LocalInputFile instance. - */ - override def toInputFile: LocalInputFile = new LocalInputFile(path) -} - -/** - * LocalPositionOutputStream is a PositionOutputStream implementation for writing to local files. - * - Tracks the write position to support Iceberg's file IO abstractions. - * - Ensures parent directories are created before writing. - * - * @param path the local file path. - * @param append whether to append to the file (if it exists). - */ -class LocalPositionOutputStream(path: String, append: Boolean) extends PositionOutputStream { - - // Ensure the parent directories exist - private val file = new File(path) - if (!file.getParentFile.exists()) { - file.getParentFile.mkdirs() - } - - private val outputStream = new FileOutputStream(file, append) - private var position: Long = 0 - - /** - * Writes a single byte to the file and updates the position. - * - * @param b the byte to write. - */ - override def write(b: Int): Unit = { - outputStream.write(b) - position += 1 - } - - /** - * Writes a sequence of bytes to the file and updates the position. - * - * @param b the byte array containing data. - * @param off the start offset in the array. - * @param len the number of bytes to write. - */ - override def write(b: Array[Byte], off: Int, len: Int): Unit = { - outputStream.write(b, off, len) - position += len - } - - /** - * Gets the current write position in the file. - * - * @return the current position in bytes. - */ - override def getPos: Long = position - - /** - * Closes the output stream and releases any system resources. - */ - override def close(): Unit = outputStream.close() -} diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 52e7a681241..89c7a7143a0 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -1,7 +1,7 @@ package edu.uci.ics.amber.util import edu.uci.ics.amber.core.storage.StorageConfig -import edu.uci.ics.amber.core.storage.result.iceberg.fileio.LocalFileIO +import edu.uci.ics.amber.core.storage.result.iceberg.LocalFileIO import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} import org.apache.iceberg.types.Types From e1059138a2376e553837384ced5f818a2b4a1445 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Mon, 30 Dec 2024 10:36:49 -0800 Subject: [PATCH 37/49] merge master --- .../scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala index 795939f1ea0..1e5deca6766 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/workflow/WorkflowCompiler.scala @@ -80,9 +80,9 @@ class WorkflowCompiler( val storageKey = OpResultStorage.createStorageKey(physicalOp.id.logicalOpId, outputPortId) - // Determine the storage type, defaulting to memory for large HTML visualizations + // Determine the storage type, defaulting to iceberg for large HTML visualizations val storageType = - if (outputPort.mode == SINGLE_SNAPSHOT) OpResultStorage.MEMORY + if (outputPort.mode == SINGLE_SNAPSHOT) OpResultStorage.ICEBERG else OpResultStorage.defaultStorageMode if (!storage.contains(storageKey)) { From 9b69f5910f9eb1cb322ea40a40fe8b2d0f901496 Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 12:48:20 -0800 Subject: [PATCH 38/49] cleanup the config --- .../src/main/resources/storage-config.yaml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/workflow-core/src/main/resources/storage-config.yaml b/core/workflow-core/src/main/resources/storage-config.yaml index da7a99d2772..171a374552d 100644 --- a/core/workflow-core/src/main/resources/storage-config.yaml +++ b/core/workflow-core/src/main/resources/storage-config.yaml @@ -7,9 +7,9 @@ storage: iceberg: catalog: jdbc: # currently we only support storing catalog info via jdbc, i.e. https://iceberg.apache.org/docs/1.7.1/jdbc/ - url: "jdbc:mysql://0.0.0.0:3306/texera_iceberg?serverTimezone=UTC" - username: "root" - password: "123456" + url: "jdbc:mysql://localhost:3306/texera_iceberg?serverTimezone=UTC" + username: "" + password: "" table: namespace: "operator-result" commit: @@ -22,6 +22,6 @@ storage: min-wait-ms: 100 # 0.1s max-wait-ms: 10000 # 10s jdbc: - url: "jdbc:mysql://0.0.0.0:3306/texera_db?serverTimezone=UTC" - username: "root" - password: "123456" \ No newline at end of file + url: "jdbc:mysql://localhost:3306/texera_db?serverTimezone=UTC" + username: "" + password: "" \ No newline at end of file From 9a482b100bbeccd58f8f942ea00f84d5b6aa9fed Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 13:23:07 -0800 Subject: [PATCH 39/49] cleanup the clear logic --- .../scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala index 0142a1eb105..285d3c0c22c 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala @@ -179,11 +179,10 @@ class ComputingUnitMaster extends io.dropwizard.Application[Configuration] with val storageType = collection.get("storageType").asText() val collectionName = collection.get("storageKey").asText() storageType match { - case OpResultStorage.MEMORY => + case OpResultStorage.MEMORY | OpResultStorage.ICEBERG => // rely on the server-side result cleanup logic. case OpResultStorage.MONGODB => MongoDatabaseManager.dropCollection(collectionName) - case OpResultStorage.ICEBERG => } }) } catch { From decab8d9fcdb937bae0b880b928c454a5476e2d0 Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 13:24:51 -0800 Subject: [PATCH 40/49] fmt --- .../main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala | 2 +- .../edu/uci/ics/amber/core/storage/result/OpResultStorage.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala index 285d3c0c22c..e0d82451c5a 100644 --- a/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala +++ b/core/amber/src/main/scala/edu/uci/ics/texera/web/ComputingUnitMaster.scala @@ -179,7 +179,7 @@ class ComputingUnitMaster extends io.dropwizard.Application[Configuration] with val storageType = collection.get("storageType").asText() val collectionName = collection.get("storageKey").asText() storageType match { - case OpResultStorage.MEMORY | OpResultStorage.ICEBERG => + case OpResultStorage.MEMORY | OpResultStorage.ICEBERG => // rely on the server-side result cleanup logic. case OpResultStorage.MONGODB => MongoDatabaseManager.dropCollection(collectionName) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala index def6526442f..06e94ae48ec 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala @@ -11,7 +11,6 @@ import edu.uci.ics.amber.util.IcebergUtil import org.apache.iceberg.data.Record import org.apache.iceberg.{Schema => IcebergSchema} - import java.util.concurrent.ConcurrentHashMap import scala.jdk.CollectionConverters.IteratorHasAsScala From 9cb26749f9e08222513bec7bf5b0482fc258f9cf Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 15:08:51 -0800 Subject: [PATCH 41/49] refactor the test to use the test db --- .../edu/uci/ics/texera/dao/MockTexeraDB.scala | 6 ++++++ .../core/storage/model/VirtualDocumentSpec.scala | 2 +- .../result/iceberg/IcebergDocumentSpec.scala | 15 +++++++++------ 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/core/dao/src/test/scala/edu/uci/ics/texera/dao/MockTexeraDB.scala b/core/dao/src/test/scala/edu/uci/ics/texera/dao/MockTexeraDB.scala index edfc5d0b5ca..a0d5ee7a78a 100644 --- a/core/dao/src/test/scala/edu/uci/ics/texera/dao/MockTexeraDB.scala +++ b/core/dao/src/test/scala/edu/uci/ics/texera/dao/MockTexeraDB.scala @@ -119,4 +119,10 @@ trait MockTexeraDB { sqlServerInstance.replaceDSLContext(dslContext.get) } + + def getJdbcUrl: String = getDBInstance.getConfiguration.getURL(database) + + def getJdbcUsername: String = username + + def getJdbcPassword: String = password } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 074041177dd..ad709234fc7 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -113,7 +113,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { val batchSize = allItems.length / numWriters val remainder = allItems.length % numWriters - // Create batches using a simple for loop + // Create writer's batches val itemBatches = (0 until numWriters).map { i => val start = i * batchSize + Math.min(i, remainder) val end = start + batchSize + (if (i < remainder) 1 else 0) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 0e31c51a7c8..f4a282cf27a 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -5,6 +5,7 @@ import edu.uci.ics.amber.core.storage.model.VirtualDocumentSpec import edu.uci.ics.amber.core.storage.result.iceberg.IcebergDocument import edu.uci.ics.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} import edu.uci.ics.amber.util.IcebergUtil +import edu.uci.ics.texera.dao.MockTexeraDB import org.apache.iceberg.catalog.Catalog import org.apache.iceberg.data.Record import org.apache.iceberg.{Schema => IcebergSchema} @@ -13,7 +14,7 @@ import org.apache.iceberg.catalog.TableIdentifier import java.sql.Timestamp import java.util.UUID -class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { +class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with MockTexeraDB { // Define Amber Schema with all possible attribute types val amberSchema: Schema = Schema( @@ -39,14 +40,16 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { IcebergUtil.fromRecord(record, amberSchema) // Create catalog instance + // - init the test db + initializeDBAndReplaceDSLContext() + // - create catalog using the test db's url, username and password and replace the catalog singleton val catalog: Catalog = IcebergUtil.createJdbcCatalog( "iceberg_document_test", StorageConfig.fileStorageDirectoryUri, - StorageConfig.icebergCatalogUrl, - StorageConfig.icebergCatalogUsername, - StorageConfig.icebergCatalogPassword + getJdbcUrl, + getJdbcUsername, + getJdbcPassword ) - IcebergCatalogInstance.replaceInstance(catalog) val tableNamespace = "test_namespace" @@ -117,7 +120,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] { } // Generate additional tuples with random data and occasional nulls - val additionalTuples = (1 to 6000).map { i => + val additionalTuples = (1 to 10000).map { i => Tuple .builder(amberSchema) .add( From 51d8a1eb7654abcbe395697a1dfb32db0dbef7d1 Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 15:26:23 -0800 Subject: [PATCH 42/49] make the test harder --- .../storage/model/VirtualDocumentSpec.scala | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index ad709234fc7..1b9ab438c2c 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -201,7 +201,7 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } - it should "read a specific range of items correctly" in { + it should "read all items using ranges correctly" in { val allItems = generateSampleItems() // Write items @@ -210,15 +210,17 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { allItems.foreach(writer.putOne) writer.close() - // Read a specific range - val from = 5 - val until = 15 - val retrievedItems = document.getRange(from, until).toList + // Read all items using ranges + val batchSize = 15 + val ranges = allItems.indices.grouped(batchSize).toList + val retrievedItems = ranges.flatMap { range => + document.getRange(range.head, range.lastOption.getOrElse(range.head) + 1).toList + } - // Verify the retrieved range + // Verify that the retrieved items match the original items assert( - retrievedItems.size == allItems.slice(from, until).size, - s"Items in range ($from, $until) should match." + retrievedItems.toSet == allItems.toSet, + "All items should be retrieved correctly using ranges." ) } From 39b0448ddb8fd5ea2b5c05631a4b92a3e733ceed Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Mon, 30 Dec 2024 15:37:25 -0800 Subject: [PATCH 43/49] make the test more clean --- .../storage/model/VirtualDocumentSpec.scala | 44 +++--- .../result/iceberg/IcebergDocumentSpec.scala | 129 ++++++++---------- 2 files changed, 79 insertions(+), 94 deletions(-) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 1b9ab438c2c..42edb5a28dd 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -201,28 +201,28 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } - it should "read all items using ranges correctly" in { - val allItems = generateSampleItems() - - // Write items - val writer = document.writer(UUID.randomUUID().toString) - writer.open() - allItems.foreach(writer.putOne) - writer.close() - - // Read all items using ranges - val batchSize = 15 - val ranges = allItems.indices.grouped(batchSize).toList - val retrievedItems = ranges.flatMap { range => - document.getRange(range.head, range.lastOption.getOrElse(range.head) + 1).toList - } - - // Verify that the retrieved items match the original items - assert( - retrievedItems.toSet == allItems.toSet, - "All items should be retrieved correctly using ranges." - ) - } +// it should "read all items using ranges correctly" in { +// val allItems = generateSampleItems() +// +// // Write items +// val writer = document.writer(UUID.randomUUID().toString) +// writer.open() +// allItems.foreach(writer.putOne) +// writer.close() +// +// // Read all items using ranges +// val batchSize = 15 +// val ranges = allItems.indices.grouped(batchSize).toList +// val retrievedItems = ranges.flatMap { range => +// document.getRange(range.head, range.lastOption.getOrElse(range.head) + 1).toList +// } +// +// // Verify that the retrieved items match the original items +// assert( +// retrievedItems.toSet == allItems.toSet, +// "All items should be retrieved correctly using ranges." +// ) +// } /** * Generates a sample list of items for testing. diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index f4a282cf27a..f2649e1848f 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -10,57 +10,70 @@ import org.apache.iceberg.catalog.Catalog import org.apache.iceberg.data.Record import org.apache.iceberg.{Schema => IcebergSchema} import org.apache.iceberg.catalog.TableIdentifier +import org.scalatest.BeforeAndAfterAll import java.sql.Timestamp import java.util.UUID -class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with MockTexeraDB { - - // Define Amber Schema with all possible attribute types - val amberSchema: Schema = Schema( - List( - new Attribute("col-string", AttributeType.STRING), - new Attribute("col-int", AttributeType.INTEGER), - new Attribute("col-bool", AttributeType.BOOLEAN), - new Attribute("col-long", AttributeType.LONG), - new Attribute("col-double", AttributeType.DOUBLE), - new Attribute("col-timestamp", AttributeType.TIMESTAMP), - new Attribute("col-binary", AttributeType.BINARY) - ) - ) - - // Define Iceberg Schema - val icebergSchema: IcebergSchema = IcebergUtil.toIcebergSchema(amberSchema) - - // Serialization function: Tuple -> Record - val serde: Tuple => Record = tuple => IcebergUtil.toGenericRecord(tuple) - - // Deserialization function: Record -> Tuple - val deserde: (IcebergSchema, Record) => Tuple = (schema, record) => - IcebergUtil.fromRecord(record, amberSchema) - - // Create catalog instance - // - init the test db - initializeDBAndReplaceDSLContext() - // - create catalog using the test db's url, username and password and replace the catalog singleton - val catalog: Catalog = IcebergUtil.createJdbcCatalog( - "iceberg_document_test", - StorageConfig.fileStorageDirectoryUri, - getJdbcUrl, - getJdbcUsername, - getJdbcPassword - ) - IcebergCatalogInstance.replaceInstance(catalog) +class IcebergDocumentSpec + extends VirtualDocumentSpec[Tuple] + with MockTexeraDB + with BeforeAndAfterAll { + var amberSchema: Schema = _ + var icebergSchema: IcebergSchema = _ + var serde: Tuple => Record = _ + var deserde: (IcebergSchema, Record) => Tuple = _ + var catalog: Catalog = _ val tableNamespace = "test_namespace" var tableName: String = _ + override def beforeAll(): Unit = { + super.beforeAll() + + // Initialize Amber Schema with all possible attribute types + amberSchema = Schema( + List( + new Attribute("col-string", AttributeType.STRING), + new Attribute("col-int", AttributeType.INTEGER), + new Attribute("col-bool", AttributeType.BOOLEAN), + new Attribute("col-long", AttributeType.LONG), + new Attribute("col-double", AttributeType.DOUBLE), + new Attribute("col-timestamp", AttributeType.TIMESTAMP), + new Attribute("col-binary", AttributeType.BINARY) + ) + ) + + // Initialize Iceberg Schema + icebergSchema = IcebergUtil.toIcebergSchema(amberSchema) + + // Initialize serialization and deserialization functions + serde = tuple => IcebergUtil.toGenericRecord(tuple) + deserde = (schema, record) => IcebergUtil.fromRecord(record, amberSchema) + + // Initialize the test database and create the Iceberg catalog + initializeDBAndReplaceDSLContext() + catalog = IcebergUtil.createJdbcCatalog( + "iceberg_document_test", + StorageConfig.fileStorageDirectoryUri, + getJdbcUrl, + getJdbcUsername, + getJdbcPassword + ) + IcebergCatalogInstance.replaceInstance(catalog) + } + override def beforeEach(): Unit = { // Generate a unique table name for each test tableName = s"test_table_${UUID.randomUUID().toString.replace("-", "")}" super.beforeEach() } + override def afterAll(): Unit = { + shutdownDB() + super.afterAll() + } + // Implementation of getDocument override def getDocument: IcebergDocument[Tuple] = { new IcebergDocument[Tuple]( @@ -72,7 +85,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with MockTexeraDB { ) } - // Implementation of isDocumentClearedgetSam + // Implementation of isDocumentCleared override def isDocumentCleared: Boolean = { val identifier = TableIdentifier.of(tableNamespace, tableName) !catalog.tableExists(identifier) @@ -112,58 +125,30 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with MockTexeraDB { .build() ) - // Function to generate random binary data def generateRandomBinary(size: Int): Array[Byte] = { val array = new Array[Byte](size) scala.util.Random.nextBytes(array) array } - // Generate additional tuples with random data and occasional nulls val additionalTuples = (1 to 10000).map { i => Tuple .builder(amberSchema) - .add( - "col-string", - AttributeType.STRING, - if (i % 7 == 0) null else s"Generated String $i" - ) - .add( - "col-int", - AttributeType.INTEGER, - if (i % 5 == 0) null else i - ) - .add( - "col-bool", - AttributeType.BOOLEAN, - if (i % 6 == 0) null else i % 2 == 0 - ) - .add( - "col-long", - AttributeType.LONG, - if (i % 4 == 0) null else i.toLong * 1000000L - ) - .add( - "col-double", - AttributeType.DOUBLE, - if (i % 3 == 0) null else i * 0.12345 - ) + .add("col-string", AttributeType.STRING, if (i % 7 == 0) null else s"Generated String $i") + .add("col-int", AttributeType.INTEGER, if (i % 5 == 0) null else i) + .add("col-bool", AttributeType.BOOLEAN, if (i % 6 == 0) null else i % 2 == 0) + .add("col-long", AttributeType.LONG, if (i % 4 == 0) null else i.toLong * 1000000L) + .add("col-double", AttributeType.DOUBLE, if (i % 3 == 0) null else i * 0.12345) .add( "col-timestamp", AttributeType.TIMESTAMP, if (i % 8 == 0) null else new Timestamp(System.currentTimeMillis() + i * 1000L) ) - .add( - "col-binary", - AttributeType.BINARY, - if (i % 9 == 0) null - else generateRandomBinary(scala.util.Random.nextInt(10) + 1) - ) + .add("col-binary", AttributeType.BINARY, if (i % 9 == 0) null else generateRandomBinary(10)) .build() } - // Combine the base tuples with the generated tuples baseTuples ++ additionalTuples } } From 73106dd4d06ab748e5078ede0f0ceceee069cb76 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Tue, 31 Dec 2024 16:25:45 -0800 Subject: [PATCH 44/49] incorporate worker idx to sink --- .../worker/promisehandlers/InitializeExecutorHandler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala index c727f422da9..d5f4bc1064b 100644 --- a/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala +++ b/core/amber/src/main/scala/edu/uci/ics/amber/engine/architecture/worker/promisehandlers/InitializeExecutorHandler.scala @@ -28,6 +28,7 @@ trait InitializeExecutorHandler { case OpExecWithCode(code, _) => ExecFactory.newExecFromJavaCode(code) case OpExecSink(storageKey, workflowIdentity, outputMode) => new ProgressiveSinkOpExec( + workerIdx, outputMode, storageKey, workflowIdentity From a2e53b57f15d7a8b215b3fde7b00f9f75ed4e5c4 Mon Sep 17 00:00:00 2001 From: Jiadong Bai Date: Tue, 31 Dec 2024 17:35:28 -0800 Subject: [PATCH 45/49] add format version and row lineage to the iceberg table --- .../src/main/resources/storage-config.yaml | 11 ++++++----- .../uci/ics/amber/core/storage/StorageConfig.scala | 6 ++++++ .../scala/edu/uci/ics/amber/util/IcebergUtil.scala | 4 +++- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/core/workflow-core/src/main/resources/storage-config.yaml b/core/workflow-core/src/main/resources/storage-config.yaml index 171a374552d..b567fc331b3 100644 --- a/core/workflow-core/src/main/resources/storage-config.yaml +++ b/core/workflow-core/src/main/resources/storage-config.yaml @@ -1,5 +1,5 @@ storage: - result-storage-mode: iceberg + result-storage-mode: memory mongodb: url: "mongodb://localhost:27017" database: "texera_storage" @@ -8,9 +8,10 @@ storage: catalog: jdbc: # currently we only support storing catalog info via jdbc, i.e. https://iceberg.apache.org/docs/1.7.1/jdbc/ url: "jdbc:mysql://localhost:3306/texera_iceberg?serverTimezone=UTC" - username: "" - password: "" + username: "root" + password: "123456" table: + format-version: 3 namespace: "operator-result" commit: batch-size: 4096 # decide the buffer size of our IcebergTableWriter @@ -23,5 +24,5 @@ storage: max-wait-ms: 10000 # 10s jdbc: url: "jdbc:mysql://localhost:3306/texera_db?serverTimezone=UTC" - username: "" - password: "" \ No newline at end of file + username: "root" + password: "123456" \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index c71b94a4757..70ad69ea5c2 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -89,6 +89,12 @@ object StorageConfig { .asInstanceOf[Map[String, Any]]("password") .asInstanceOf[String] + val icebergTableFormatVersion: Int = conf("storage") + .asInstanceOf[Map[String, Any]]("iceberg") + .asInstanceOf[Map[String, Any]]("table") + .asInstanceOf[Map[String, Any]]("format-version") + .asInstanceOf[Int] + val icebergTableNamespace: String = conf("storage") .asInstanceOf[Map[String, Any]]("iceberg") .asInstanceOf[Map[String, Any]]("table") diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 89c7a7143a0..85cfe068796 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -81,7 +81,9 @@ object IcebergUtil { val tableProperties = Map( TableProperties.COMMIT_NUM_RETRIES -> StorageConfig.icebergTableCommitNumRetries.toString, TableProperties.COMMIT_MAX_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMaxRetryWaitMs.toString, - TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString + TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString, + TableProperties.FORMAT_VERSION -> StorageConfig.icebergTableFormatVersion.toString, + "row-lineage" -> true.toString ) val identifier = TableIdentifier.of(tableNamespace, tableName) if (catalog.tableExists(identifier) && overrideIfExists) { From f54e38c24c9ec58acc49323fc74b5937760534bf Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Wed, 1 Jan 2025 15:11:34 -0800 Subject: [PATCH 46/49] Revert "add format version and row lineage to the iceberg table" This reverts commit a2e53b57f15d7a8b215b3fde7b00f9f75ed4e5c4. --- .../src/main/resources/storage-config.yaml | 11 +++++------ .../uci/ics/amber/core/storage/StorageConfig.scala | 6 ------ .../scala/edu/uci/ics/amber/util/IcebergUtil.scala | 4 +--- 3 files changed, 6 insertions(+), 15 deletions(-) diff --git a/core/workflow-core/src/main/resources/storage-config.yaml b/core/workflow-core/src/main/resources/storage-config.yaml index b567fc331b3..171a374552d 100644 --- a/core/workflow-core/src/main/resources/storage-config.yaml +++ b/core/workflow-core/src/main/resources/storage-config.yaml @@ -1,5 +1,5 @@ storage: - result-storage-mode: memory + result-storage-mode: iceberg mongodb: url: "mongodb://localhost:27017" database: "texera_storage" @@ -8,10 +8,9 @@ storage: catalog: jdbc: # currently we only support storing catalog info via jdbc, i.e. https://iceberg.apache.org/docs/1.7.1/jdbc/ url: "jdbc:mysql://localhost:3306/texera_iceberg?serverTimezone=UTC" - username: "root" - password: "123456" + username: "" + password: "" table: - format-version: 3 namespace: "operator-result" commit: batch-size: 4096 # decide the buffer size of our IcebergTableWriter @@ -24,5 +23,5 @@ storage: max-wait-ms: 10000 # 10s jdbc: url: "jdbc:mysql://localhost:3306/texera_db?serverTimezone=UTC" - username: "root" - password: "123456" \ No newline at end of file + username: "" + password: "" \ No newline at end of file diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala index 70ad69ea5c2..c71b94a4757 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/StorageConfig.scala @@ -89,12 +89,6 @@ object StorageConfig { .asInstanceOf[Map[String, Any]]("password") .asInstanceOf[String] - val icebergTableFormatVersion: Int = conf("storage") - .asInstanceOf[Map[String, Any]]("iceberg") - .asInstanceOf[Map[String, Any]]("table") - .asInstanceOf[Map[String, Any]]("format-version") - .asInstanceOf[Int] - val icebergTableNamespace: String = conf("storage") .asInstanceOf[Map[String, Any]]("iceberg") .asInstanceOf[Map[String, Any]]("table") diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 85cfe068796..89c7a7143a0 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -81,9 +81,7 @@ object IcebergUtil { val tableProperties = Map( TableProperties.COMMIT_NUM_RETRIES -> StorageConfig.icebergTableCommitNumRetries.toString, TableProperties.COMMIT_MAX_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMaxRetryWaitMs.toString, - TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString, - TableProperties.FORMAT_VERSION -> StorageConfig.icebergTableFormatVersion.toString, - "row-lineage" -> true.toString + TableProperties.COMMIT_MIN_RETRY_WAIT_MS -> StorageConfig.icebergTableCommitMinRetryWaitMs.toString ) val identifier = TableIdentifier.of(tableNamespace, tableName) if (catalog.tableExists(identifier) && overrideIfExists) { From 71768644bc4b85271795e9e873ab99edd64e2551 Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Wed, 1 Jan 2025 15:12:39 -0800 Subject: [PATCH 47/49] fix iceberg util spec --- .../test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala index 9549c22a0ac..df07166ff3f 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala @@ -13,8 +13,7 @@ import scala.jdk.CollectionConverters._ class IcebergUtilSpec extends AnyFlatSpec { - val texeraSchema: Schema = Schema - .builder() + val texeraSchema: Schema = Schema() .add("test-1", AttributeType.INTEGER) .add("test-2", AttributeType.LONG) .add("test-3", AttributeType.BOOLEAN) @@ -22,7 +21,6 @@ class IcebergUtilSpec extends AnyFlatSpec { .add("test-5", AttributeType.TIMESTAMP) .add("test-6", AttributeType.STRING) .add("test-7", AttributeType.BINARY) - .build() val icebergSchema: IcebergSchema = new IcebergSchema( List( From 76dd31c69971f3005b00fd56f5f60ebc4aa30b2b Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Wed, 1 Jan 2025 16:14:51 -0800 Subject: [PATCH 48/49] try to add the record id --- .../core/storage/result/OpResultStorage.scala | 2 +- .../result/iceberg/IcebergDocument.scala | 19 +++++++++---- .../result/iceberg/IcebergTableWriter.scala | 22 +++++++++------ .../edu/uci/ics/amber/util/IcebergUtil.scala | 28 +++++++++++++++++-- .../result/iceberg/IcebergDocumentSpec.scala | 4 +-- .../uci/ics/amber/util/IcebergUtilSpec.scala | 7 +++-- 6 files changed, 61 insertions(+), 21 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala index 06e94ae48ec..eb295950164 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/OpResultStorage.scala @@ -132,7 +132,7 @@ class OpResultStorage extends Serializable with LazyLogging { } } else { val icebergSchema = IcebergUtil.toIcebergSchema(schema) - val serde: Tuple => Record = tuple => IcebergUtil.toGenericRecord(tuple) + val serde: (IcebergSchema, Tuple) => Record = IcebergUtil.toGenericRecord val deserde: (IcebergSchema, Record) => Tuple = (_, record) => IcebergUtil.fromRecord(record, schema) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 395c0cb1bee..91753bfd559 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -4,11 +4,12 @@ import edu.uci.ics.amber.core.storage.IcebergCatalogInstance import edu.uci.ics.amber.core.storage.model.{BufferedItemWriter, VirtualDocument} import edu.uci.ics.amber.core.storage.util.StorageUtil.{withLock, withReadLock, withWriteLock} import edu.uci.ics.amber.util.IcebergUtil -import org.apache.iceberg.{Snapshot, Table} +import org.apache.iceberg.{Schema, Table} import org.apache.iceberg.catalog.{Catalog, TableIdentifier} import org.apache.iceberg.data.{IcebergGenerics, Record} import org.apache.iceberg.exceptions.NoSuchTableException import org.apache.iceberg.io.CloseableIterable +import org.apache.iceberg.types.Types import java.net.URI import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock} @@ -32,7 +33,7 @@ class IcebergDocument[T >: Null <: AnyRef]( val tableNamespace: String, val tableName: String, val tableSchema: org.apache.iceberg.Schema, - val serde: T => Record, + val serde: (org.apache.iceberg.Schema, T) => Record, val deserde: (org.apache.iceberg.Schema, Record) => T ) extends VirtualDocument[T] { @@ -40,13 +41,21 @@ class IcebergDocument[T >: Null <: AnyRef]( @transient lazy val catalog: Catalog = IcebergCatalogInstance.getInstance() + // Add the recordId field to the schema + private val augmentedSchema: org.apache.iceberg.Schema = + IcebergUtil.addFieldToSchema( + tableSchema, + "_record_id", + Types.StringType.get() + ) + // During construction, create or override the table synchronized { IcebergUtil.createTable( catalog, tableNamespace, tableName, - tableSchema, + augmentedSchema, overrideIfExists = true ) } @@ -130,7 +139,7 @@ class IcebergDocument[T >: Null <: AnyRef]( } lastSnapshotId = currentSnapshotId - records.iterator().asScala.map(record => deserde(tableSchema, record)) + records.iterator().asScala.map(record => deserde(augmentedSchema, record)) case _ => Iterator.empty } @@ -189,7 +198,7 @@ class IcebergDocument[T >: Null <: AnyRef]( catalog, tableNamespace, tableName, - tableSchema, + augmentedSchema, serde ) } diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala index e945968b791..9021f473149 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergTableWriter.scala @@ -33,13 +33,16 @@ class IcebergTableWriter[T]( val tableNamespace: String, val tableName: String, val tableSchema: Schema, - val serde: T => Record + val serde: (org.apache.iceberg.Schema, T) => Record ) extends BufferedItemWriter[T] { // Buffer to hold items before flushing to the table private val buffer = new ArrayBuffer[T]() // Incremental filename index, incremented each time a new buffer is flushed private var filenameIdx = 0 + // Incremental record ID, incremented for each record + private var recordId = 0 + override val bufferSize: Int = StorageConfig.icebergTableCommitBatchSize // Load the Iceberg table @@ -51,8 +54,9 @@ class IcebergTableWriter[T]( /** * Open the writer and clear the buffer. */ - override def open(): Unit = + override def open(): Unit = { buffer.clear() + } /** * Add a single item to the buffer. @@ -70,8 +74,9 @@ class IcebergTableWriter[T]( * Remove a single item from the buffer. * @param item the item to remove from the buffer. */ - override def removeOne(item: T): Unit = + override def removeOne(item: T): Unit = { buffer -= item + } /** * Flush the current buffer to a new Iceberg data file. @@ -88,10 +93,6 @@ class IcebergTableWriter[T]( val outputFile: OutputFile = table.io().newOutputFile(filepath) // Create a Parquet data writer to write a new file - // This part introduces the dependency to the Hadoop. In the source code of iceberg-parquet, see the line 160 - // https://github.com/apache/iceberg/blob/main/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java - // although the file is not of type HadoopOutputFile, it still creats a Hadoop Configuration() as the - // placeholder. val dataWriter: DataWriter[Record] = Parquet .writeData(outputFile) .forTable(table) @@ -102,7 +103,12 @@ class IcebergTableWriter[T]( // Write each buffered item to the data file try { buffer.foreach { item => - val record = serde(item) + val record = serde(tableSchema, item) + + // Add the _record_id field + record.setField("_record_id", s"${writerIdentifier}_${recordId}") + recordId += 1 + dataWriter.write(record) } } finally { diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala index 89c7a7143a0..6605eb88ea4 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/util/IcebergUtil.scala @@ -158,8 +158,7 @@ object IcebergUtil { * @param tuple The custom Amber Tuple. * @return An Iceberg GenericRecord. */ - def toGenericRecord(tuple: Tuple): Record = { - val icebergSchema = toIcebergSchema(tuple.schema) + def toGenericRecord(icebergSchema: IcebergSchema, tuple: Tuple): Record = { val record = GenericRecord.create(icebergSchema) tuple.schema.getAttributes.zipWithIndex.foreach { @@ -236,4 +235,29 @@ object IcebergUtil { case _ => throw new IllegalArgumentException(s"Unsupported Iceberg type: $icebergType") } } + + /** + * Adds a new field to an existing Iceberg schema. + * - If the field already exists, it throws an IllegalArgumentException. + * + * @param schema The existing Iceberg schema. + * @param fieldName The name of the new field. + * @param fieldType The type of the new field. + * @return The updated Iceberg schema with the new field. + */ + def addFieldToSchema( + schema: IcebergSchema, + fieldName: String, + fieldType: PrimitiveType + ): IcebergSchema = { + if (schema.findField(fieldName) != null) { + throw new IllegalArgumentException(s"Field $fieldName already exists in the schema") + } + + val updatedFields = schema.columns().asScala.toSeq :+ + Types.NestedField.optional(schema.columns().size() + 1, fieldName, fieldType) + + new IcebergSchema(updatedFields.asJava) + } + } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index f2649e1848f..99bdabe73e1 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -22,7 +22,7 @@ class IcebergDocumentSpec var amberSchema: Schema = _ var icebergSchema: IcebergSchema = _ - var serde: Tuple => Record = _ + var serde: (IcebergSchema, Tuple) => Record = _ var deserde: (IcebergSchema, Record) => Tuple = _ var catalog: Catalog = _ val tableNamespace = "test_namespace" @@ -48,7 +48,7 @@ class IcebergDocumentSpec icebergSchema = IcebergUtil.toIcebergSchema(amberSchema) // Initialize serialization and deserialization functions - serde = tuple => IcebergUtil.toGenericRecord(tuple) + serde = IcebergUtil.toGenericRecord deserde = (schema, record) => IcebergUtil.fromRecord(record, amberSchema) // Initialize the test database and create the Iceberg catalog diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala index df07166ff3f..6fc07535e0b 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/util/IcebergUtilSpec.scala @@ -1,6 +1,7 @@ package edu.uci.ics.amber.util import edu.uci.ics.amber.core.tuple.{AttributeType, Schema, Tuple} +import edu.uci.ics.amber.util.IcebergUtil.toIcebergSchema import org.apache.iceberg.types.Types import org.apache.iceberg.{Schema => IcebergSchema} import org.apache.iceberg.data.GenericRecord @@ -82,7 +83,7 @@ class IcebergUtilSpec extends AnyFlatSpec { ) .build() - val record = IcebergUtil.toGenericRecord(tuple) + val record = IcebergUtil.toGenericRecord(toIcebergSchema(tuple.schema), tuple) assert(record.getField("test-1") == 42) assert(record.getField("test-2") == 123456789L) @@ -112,7 +113,7 @@ class IcebergUtilSpec extends AnyFlatSpec { ) .build() - val record = IcebergUtil.toGenericRecord(tuple) + val record = IcebergUtil.toGenericRecord(toIcebergSchema(tuple.schema), tuple) assert(record.getField("test-1") == 42) assert(record.getField("test-2") == null) @@ -142,7 +143,7 @@ class IcebergUtilSpec extends AnyFlatSpec { ) .build() - val record = IcebergUtil.toGenericRecord(tuple) + val record = IcebergUtil.toGenericRecord(toIcebergSchema(tuple.schema), tuple) assert(record.getField("test-1") == null) assert(record.getField("test-2") == null) From 31070bed4b028bfb70a93e7752c64ec54480b57d Mon Sep 17 00:00:00 2001 From: bobbai00 Date: Wed, 1 Jan 2025 23:45:18 -0800 Subject: [PATCH 49/49] try debugging the test --- .../result/iceberg/IcebergDocument.scala | 11 ++++- .../storage/model/VirtualDocumentSpec.scala | 46 ++++++++++--------- .../result/iceberg/IcebergDocumentSpec.scala | 2 +- 3 files changed, 34 insertions(+), 25 deletions(-) diff --git a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala index 91753bfd559..cf9031a5a72 100644 --- a/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala +++ b/core/workflow-core/src/main/scala/edu/uci/ics/amber/core/storage/result/iceberg/IcebergDocument.scala @@ -51,13 +51,16 @@ class IcebergDocument[T >: Null <: AnyRef]( // During construction, create or override the table synchronized { - IcebergUtil.createTable( + val table = IcebergUtil.createTable( catalog, tableNamespace, tableName, augmentedSchema, overrideIfExists = true ) + table.replaceSortOrder().asc("_record_id").commit() + print(table.sortOrders()) + print(table.properties()) } /** @@ -139,7 +142,11 @@ class IcebergDocument[T >: Null <: AnyRef]( } lastSnapshotId = currentSnapshotId - records.iterator().asScala.map(record => deserde(augmentedSchema, record)) + records.iterator().asScala.map(record => { + println(record.getField("_record_id")) + deserde(augmentedSchema, record) + } + ) case _ => Iterator.empty } diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala index 42edb5a28dd..8c01c60a5dd 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/core/storage/model/VirtualDocumentSpec.scala @@ -201,28 +201,30 @@ trait VirtualDocumentSpec[T] extends AnyFlatSpec with BeforeAndAfterEach { ) } -// it should "read all items using ranges correctly" in { -// val allItems = generateSampleItems() -// -// // Write items -// val writer = document.writer(UUID.randomUUID().toString) -// writer.open() -// allItems.foreach(writer.putOne) -// writer.close() -// -// // Read all items using ranges -// val batchSize = 15 -// val ranges = allItems.indices.grouped(batchSize).toList -// val retrievedItems = ranges.flatMap { range => -// document.getRange(range.head, range.lastOption.getOrElse(range.head) + 1).toList -// } -// -// // Verify that the retrieved items match the original items -// assert( -// retrievedItems.toSet == allItems.toSet, -// "All items should be retrieved correctly using ranges." -// ) -// } + it should "read all items using ranges correctly" in { + val allItems = generateSampleItems() + + // Write items + val writer = document.writer(UUID.randomUUID().toString) + writer.open() + allItems.foreach(writer.putOne) + writer.close() + + // Read all items using ranges + val batchSize = 500 + val ranges = allItems.indices.grouped(batchSize).toList + val retrievedItems = ranges.flatMap { range => + document.getRange(range.head, range.lastOption.getOrElse(range.head) + 1).toList + } + + assert(retrievedItems.size == allItems.size) + + // Verify that the retrieved items match the original items + assert( + retrievedItems.toSet == allItems.toSet, + "All items should be retrieved correctly using ranges." + ) + } /** * Generates a sample list of items for testing. diff --git a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 99bdabe73e1..35a1fca0b04 100644 --- a/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/core/workflow-core/src/test/scala/edu/uci/ics/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -131,7 +131,7 @@ class IcebergDocumentSpec array } - val additionalTuples = (1 to 10000).map { i => + val additionalTuples = (1 to 1000).map { i => Tuple .builder(amberSchema) .add("col-string", AttributeType.STRING, if (i % 7 == 0) null else s"Generated String $i")