diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 35dfa7a6c3499..f9def4a96b1fa 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -224,6 +224,11 @@ "Error reading delta file of : does not exist." ] }, + "CANNOT_READ_MISSING_SNAPSHOT_FILE" : { + "message" : [ + "Error reading snapshot file of : does not exist." + ] + }, "CANNOT_READ_SNAPSHOT_FILE_KEY_SIZE" : { "message" : [ "Error reading snapshot file of : key size cannot be ." @@ -239,6 +244,11 @@ "Error reading streaming state file of does not exist. If the stream job is restarted with a new or updated state operation, please create a new checkpoint location or clear the existing checkpoint location." ] }, + "SNAPSHOT_PARTITION_ID_NOT_FOUND" : { + "message" : [ + "Partition id not found for state of operator at ." + ] + }, "UNCATEGORIZED" : { "message" : [ "" @@ -3763,6 +3773,13 @@ ], "sqlState" : "42802" }, + "STATE_STORE_PROVIDER_DOES_NOT_SUPPORT_FINE_GRAINED_STATE_REPLAY" : { + "message" : [ + "The given State Store Provider does not extend org.apache.spark.sql.execution.streaming.state.SupportsFineGrainedReplay.", + "Therefore, it does not support option snapshotStartBatchId in state data source." + ], + "sqlState" : "42K06" + }, "STATE_STORE_UNSUPPORTED_OPERATION" : { "message" : [ " operation not supported with " diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala index 1a8f444042c23..e2724cb59754d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala @@ -116,12 +116,16 @@ case class StateSourceOptions( batchId: Long, operatorId: Int, storeName: String, - joinSide: JoinSideValues) { + joinSide: JoinSideValues, + snapshotStartBatchId: Option[Long], + snapshotPartitionId: Option[Int]) { def stateCheckpointLocation: Path = new Path(resolvedCpLocation, DIR_NAME_STATE) override def toString: String = { s"StateSourceOptions(checkpointLocation=$resolvedCpLocation, batchId=$batchId, " + - s"operatorId=$operatorId, storeName=$storeName, joinSide=$joinSide)" + s"operatorId=$operatorId, storeName=$storeName, joinSide=$joinSide, " + + s"snapshotStartBatchId=${snapshotStartBatchId.getOrElse("None")}, " + + s"snapshotPartitionId=${snapshotPartitionId.getOrElse("None")})" } } @@ -131,6 +135,8 @@ object StateSourceOptions extends DataSourceOptions { val OPERATOR_ID = newOption("operatorId") val STORE_NAME = newOption("storeName") val JOIN_SIDE = newOption("joinSide") + val SNAPSHOT_START_BATCH_ID = newOption("snapshotStartBatchId") + val SNAPSHOT_PARTITION_ID = newOption("snapshotPartitionId") object JoinSideValues extends Enumeration { type JoinSideValues = Value @@ -190,7 +196,30 @@ object StateSourceOptions extends DataSourceOptions { throw StateDataSourceErrors.conflictOptions(Seq(JOIN_SIDE, STORE_NAME)) } - StateSourceOptions(resolvedCpLocation, batchId, operatorId, storeName, joinSide) + val snapshotStartBatchId = Option(options.get(SNAPSHOT_START_BATCH_ID)).map(_.toLong) + if (snapshotStartBatchId.exists(_ < 0)) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_START_BATCH_ID) + } else if (snapshotStartBatchId.exists(_ > batchId)) { + throw StateDataSourceErrors.invalidOptionValue( + SNAPSHOT_START_BATCH_ID, s"value should be less than or equal to $batchId") + } + + val snapshotPartitionId = Option(options.get(SNAPSHOT_PARTITION_ID)).map(_.toInt) + if (snapshotPartitionId.exists(_ < 0)) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_PARTITION_ID) + } + + // both snapshotPartitionId and snapshotStartBatchId are required at the same time, because + // each partition may have different checkpoint status + if (snapshotPartitionId.isDefined && snapshotStartBatchId.isEmpty) { + throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_START_BATCH_ID) + } else if (snapshotPartitionId.isEmpty && snapshotStartBatchId.isDefined) { + throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_PARTITION_ID) + } + + StateSourceOptions( + resolvedCpLocation, batchId, operatorId, storeName, + joinSide, snapshotStartBatchId, snapshotPartitionId) } private def resolvedCheckpointLocation( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala index bbfe3a3f373ec..f09a2763031e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataPartitionReader import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, ReadStateStore, StateStoreConf, StateStoreId, StateStoreProvider, StateStoreProviderId} +import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -93,7 +93,19 @@ class StatePartitionReader( } private lazy val store: ReadStateStore = { - provider.getReadStore(partition.sourceOptions.batchId + 1) + partition.sourceOptions.snapshotStartBatchId match { + case None => provider.getReadStore(partition.sourceOptions.batchId + 1) + + case Some(snapshotStartBatchId) => + if (!provider.isInstanceOf[SupportsFineGrainedReplay]) { + throw StateStoreErrors.stateStoreProviderDoesNotSupportFineGrainedReplay( + provider.getClass.toString) + } + provider.asInstanceOf[SupportsFineGrainedReplay] + .replayReadStateFromSnapshot( + snapshotStartBatchId + 1, + partition.sourceOptions.batchId + 1) + } } private lazy val iter: Iterator[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala index 0d69bf708e94f..ffcbcd0872e10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} -import org.apache.spark.sql.execution.streaming.state.StateStoreConf +import org.apache.spark.sql.execution.streaming.state.{StateStoreConf, StateStoreErrors} import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -81,9 +81,20 @@ class StateScan( assert((tail - head + 1) == partitionNums.length, s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}") - partitionNums.map { - pn => new StateStoreInputPartition(pn, queryId, sourceOptions) - }.toArray + sourceOptions.snapshotPartitionId match { + case None => partitionNums.map { pn => + new StateStoreInputPartition(pn, queryId, sourceOptions) + }.toArray + + case Some(snapshotPartitionId) => + if (partitionNums.contains(snapshotPartitionId)) { + Array(new StateStoreInputPartition(snapshotPartitionId, queryId, sourceOptions)) + } else { + throw StateStoreErrors.stateStoreSnapshotPartitionNotFound( + snapshotPartitionId, sourceOptions.operatorId, + sourceOptions.stateCheckpointLocation.toString) + } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala index 824968e709baf..dbd39f519e500 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala @@ -49,16 +49,21 @@ class StateTable( } override def name(): String = { - val desc = s"StateTable " + + var desc = s"StateTable " + s"[stateCkptLocation=${sourceOptions.stateCheckpointLocation}]" + s"[batchId=${sourceOptions.batchId}][operatorId=${sourceOptions.operatorId}]" + s"[storeName=${sourceOptions.storeName}]" if (sourceOptions.joinSide != JoinSideValues.none) { - desc + s"[joinSide=${sourceOptions.joinSide}]" - } else { - desc + desc += s"[joinSide=${sourceOptions.joinSide}]" + } + if (sourceOptions.snapshotStartBatchId.isDefined) { + desc += s"[snapshotStartBatchId=${sourceOptions.snapshotStartBatchId}]" + } + if (sourceOptions.snapshotPartitionId.isDefined) { + desc += s"[snapshotPartitionId=${sourceOptions.snapshotPartitionId}]" } + desc } override def capabilities(): util.Set[TableCapability] = CAPABILITY diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala index e5a5dddefef5b..91f42db46dfb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala @@ -116,7 +116,8 @@ class StreamStreamJoinStatePartitionReader( partitionId = partition.partition, formatVersion, skippedNullValueCount = None, - useStateStoreCoordinator = false + useStateStoreCoordinator = false, + snapshotStartVersion = partition.sourceOptions.snapshotStartBatchId.map(_ + 1) ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 543cd74c489d0..c4a41ceb4caf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -71,7 +71,8 @@ import org.apache.spark.util.ArrayImplicits._ * to ensure re-executed RDD operations re-apply updates on the correct past version of the * store. */ -private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with Logging { +private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with Logging + with SupportsFineGrainedReplay { private val providerName = "HDFSBackedStateStoreProvider" @@ -683,6 +684,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with } } + /** + * Try to read the snapshot file. If the snapshot file is not available, return [[None]]. + * + * @param version the version of the snapshot file + */ private def readSnapshotFile(version: Long): Option[HDFSBackedStateStoreMap] = { val fileToRead = snapshotFile(version) val map = HDFSBackedStateStoreMap.create(keySchema, numColsPrefixKey) @@ -883,4 +889,93 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with throw new IllegalStateException(msg) } } + + /** + * Get the state store of endVersion by applying delta files on the snapshot of snapshotVersion. + * If snapshot for snapshotVersion does not exist, an error will be thrown. + * + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + * @return [[HDFSBackedStateStore]] + */ + override def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore = { + val newMap = replayLoadedMapFromSnapshot(snapshotVersion, endVersion) + logInfo(log"Retrieved snapshot at version " + + log"${MDC(LogKeys.STATE_STORE_VERSION, snapshotVersion)} and apply delta files to version " + + log"${MDC(LogKeys.STATE_STORE_VERSION, endVersion)} of " + + log"${MDC(LogKeys.STATE_STORE_PROVIDER, HDFSBackedStateStoreProvider.this)} for update") + new HDFSBackedStateStore(endVersion, newMap) + } + + /** + * Get the state store of endVersion for reading by applying delta files on the snapshot of + * snapshotVersion. If snapshot for snapshotVersion does not exist, an error will be thrown. + * + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + * @return [[HDFSBackedReadStateStore]] + */ + override def replayReadStateFromSnapshot(snapshotVersion: Long, endVersion: Long): + ReadStateStore = { + val newMap = replayLoadedMapFromSnapshot(snapshotVersion, endVersion) + logInfo(log"Retrieved snapshot at version " + + log"${MDC(LogKeys.STATE_STORE_VERSION, snapshotVersion)} and apply delta files to version " + + log"${MDC(LogKeys.STATE_STORE_VERSION, endVersion)} of " + + log"${MDC(LogKeys.STATE_STORE_PROVIDER, HDFSBackedStateStoreProvider.this)} for read-only") + new HDFSBackedReadStateStore(endVersion, newMap) + } + + /** + * Construct the state map at endVersion from snapshot of version snapshotVersion. + * Returns a new [[HDFSBackedStateStoreMap]] + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + */ + private def replayLoadedMapFromSnapshot(snapshotVersion: Long, endVersion: Long): + HDFSBackedStateStoreMap = synchronized { + try { + if (snapshotVersion < 1) { + throw QueryExecutionErrors.unexpectedStateStoreVersion(snapshotVersion) + } + if (endVersion < snapshotVersion) { + throw QueryExecutionErrors.unexpectedStateStoreVersion(endVersion) + } + + val newMap = HDFSBackedStateStoreMap.create(keySchema, numColsPrefixKey) + newMap.putAll(constructMapFromSnapshot(snapshotVersion, endVersion)) + + newMap + } + catch { + case e: Throwable => throw QueryExecutionErrors.cannotLoadStore(e) + } + } + + private def constructMapFromSnapshot(snapshotVersion: Long, endVersion: Long): + HDFSBackedStateStoreMap = { + val (result, elapsedMs) = Utils.timeTakenMs { + val startVersionMap = synchronized { Option(loadedMaps.get(snapshotVersion)) } match { + case Some(value) => Option(value) + case None => readSnapshotFile(snapshotVersion) + } + if (startVersionMap.isEmpty) { + throw StateStoreErrors.stateStoreSnapshotFileNotFound( + snapshotFile(snapshotVersion).toString, toString()) + } + + // Load all the deltas from the version after the start version up to the end version. + val resultMap = HDFSBackedStateStoreMap.create(keySchema, numColsPrefixKey) + resultMap.putAll(startVersionMap.get) + for (deltaVersion <- snapshotVersion + 1 to endVersion) { + updateFromDeltaFile(deltaVersion, resultMap) + } + + resultMap + } + + logDebug(s"Loading snapshot at version $snapshotVersion and apply delta files to version " + + s"$endVersion takes $elapsedMs ms.") + + result + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 088242b4246e5..49d7a536b0018 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -226,6 +226,80 @@ class RocksDB( this } + /** + * Load from the start snapshot version and apply all the changelog records to reach the + * end version. Note that this will copy all the necessary files from DFS to local disk as needed, + * and possibly restart the native RocksDB instance. + * + * @param snapshotVersion version of the snapshot to start with + * @param endVersion end version + * @return A RocksDB instance loaded with the state endVersion replayed from snapshotVersion. + * Note that the instance will be read-only since this method is only used in State Data + * Source. + */ + def loadFromSnapshot(snapshotVersion: Long, endVersion: Long): RocksDB = { + assert(snapshotVersion >= 0 && endVersion >= snapshotVersion) + acquire(LoadStore) + recordedMetrics = None + logInfo( + log"Loading snapshot at version ${MDC(LogKeys.VERSION_NUM, snapshotVersion)} and apply " + + log"changelog files to version ${MDC(LogKeys.VERSION_NUM, endVersion)}.") + try { + replayFromCheckpoint(snapshotVersion, endVersion) + + logInfo( + log"Loaded snapshot at version ${MDC(LogKeys.VERSION_NUM, snapshotVersion)} and apply " + + log"changelog files to version ${MDC(LogKeys.VERSION_NUM, endVersion)}.") + } catch { + case t: Throwable => + loadedVersion = -1 // invalidate loaded data + throw t + } + this + } + + /** + * Load from the start checkpoint version and apply all the changelog records to reach the + * end version. + * If the start version does not exist, it will throw an exception. + * + * @param snapshotVersion start checkpoint version + * @param endVersion end version + */ + private def replayFromCheckpoint(snapshotVersion: Long, endVersion: Long): Any = { + closeDB() + val metadata = fileManager.loadCheckpointFromDfs(snapshotVersion, workingDir) + loadedVersion = snapshotVersion + + // reset last snapshot version + if (lastSnapshotVersion > snapshotVersion) { + // discard any newer snapshots + lastSnapshotVersion = 0L + latestSnapshot = None + } + openDB() + + numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { + // we don't track the total number of rows - discard the number being track + -1L + } else if (metadata.numKeys < 0) { + // we track the total number of rows, but the snapshot doesn't have tracking number + // need to count keys now + countKeys() + } else { + metadata.numKeys + } + if (loadedVersion != endVersion) replayChangelog(endVersion) + // After changelog replay the numKeysOnWritingVersion will be updated to + // the correct number of keys in the loaded version. + numKeysOnLoadedVersion = numKeysOnWritingVersion + fileManagerMetrics = fileManager.latestLoadCheckpointMetrics + + if (conf.resetStatsOnLoad) { + nativeStats.reset + } + } + /** * Replay change log from the loaded version to the target version. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index e7fc9f56dd9eb..a555f9a40044a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -32,7 +32,8 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils private[sql] class RocksDBStateStoreProvider - extends StateStoreProvider with Logging with Closeable { + extends StateStoreProvider with Logging with Closeable + with SupportsFineGrainedReplay { import RocksDBStateStoreProvider._ class RocksDBStateStore(lastVersion: Long) extends StateStore { @@ -367,6 +368,30 @@ private[sql] class RocksDBStateStoreProvider private def verify(condition: => Boolean, msg: String): Unit = { if (!condition) { throw new IllegalStateException(msg) } } + + /** + * Get the state store of endVersion by applying delta files on the snapshot of snapshotVersion. + * If snapshot for snapshotVersion does not exist, an error will be thrown. + * + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + * @return [[StateStore]] + */ + override def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore = { + try { + if (snapshotVersion < 1) { + throw QueryExecutionErrors.unexpectedStateStoreVersion(snapshotVersion) + } + if (endVersion < snapshotVersion) { + throw QueryExecutionErrors.unexpectedStateStoreVersion(endVersion) + } + rocksDB.loadFromSnapshot(snapshotVersion, endVersion) + new RocksDBStateStore(endVersion) + } + catch { + case e: Throwable => throw QueryExecutionErrors.cannotLoadStore(e) + } + } } object RocksDBStateStoreProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 2f9ce2c236f4e..b8739fb8e4cf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -439,6 +439,39 @@ object StateStoreProvider { } } +/** + * This is an optional trait to be implemented by [[StateStoreProvider]]s that can read fine + * grained state data which is replayed from a specific snapshot version. It is used by the + * snapshotStartBatchId option in state data source. + */ +trait SupportsFineGrainedReplay { + + /** + * Return an instance of [[StateStore]] representing state data of the given version. + * The State Store will be constructed from the snapshot at snapshotVersion, and applying delta + * files up to the endVersion. If there is no snapshot file at snapshotVersion, an exception will + * be thrown. + * + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + */ + def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore + + /** + * Return an instance of [[ReadStateStore]] representing state data of the given version. + * The State Store will be constructed from the snapshot at snapshotVersion, and applying delta + * files up to the endVersion. If there is no snapshot file at snapshotVersion, an exception will + * be thrown. + * Only implement this if there is read-only optimization for the state store. + * + * @param snapshotVersion checkpoint version of the snapshot to start with + * @param endVersion checkpoint version to end with + */ + def replayReadStateFromSnapshot(snapshotVersion: Long, endVersion: Long): ReadStateStore = { + new WrappedReadStateStore(replayStateFromSnapshot(snapshotVersion, endVersion)) + } +} + /** * Unique identifier for a provider, used to identify when providers can be reused. * Note that `queryRunId` is used uniquely identify a provider, so that the same provider diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala index 205e093e755d5..0844a87739765 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala @@ -149,6 +149,22 @@ object StateStoreErrors { newValueSchema: String): StateStoreValueSchemaNotCompatible = { new StateStoreValueSchemaNotCompatible(storedValueSchema, newValueSchema) } + + def stateStoreSnapshotFileNotFound(fileToRead: String, clazz: String): + StateStoreSnapshotFileNotFound = { + new StateStoreSnapshotFileNotFound(fileToRead, clazz) + } + + def stateStoreSnapshotPartitionNotFound( + snapshotPartitionId: Long, operatorId: Int, checkpointLocation: String): + StateStoreSnapshotPartitionNotFound = { + new StateStoreSnapshotPartitionNotFound(snapshotPartitionId, operatorId, checkpointLocation) + } + + def stateStoreProviderDoesNotSupportFineGrainedReplay(inputClass: String): + StateStoreProviderDoesNotSupportFineGrainedReplay = { + new StateStoreProviderDoesNotSupportFineGrainedReplay(inputClass) + } } class StateStoreMultipleColumnFamiliesNotSupportedException(stateStoreProvider: String) @@ -255,6 +271,22 @@ class StateStoreValueSchemaNotCompatible( "storedValueSchema" -> storedValueSchema, "newValueSchema" -> newValueSchema)) +class StateStoreSnapshotFileNotFound(fileToRead: String, clazz: String) + extends SparkRuntimeException( + errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_MISSING_SNAPSHOT_FILE", + messageParameters = Map( + "fileToRead" -> fileToRead, + "clazz" -> clazz)) + +class StateStoreSnapshotPartitionNotFound( + snapshotPartitionId: Long, operatorId: Int, checkpointLocation: String) + extends SparkRuntimeException( + errorClass = "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND", + messageParameters = Map( + "snapshotPartitionId" -> snapshotPartitionId.toString, + "operatorId" -> operatorId.toString, + "checkpointLocation" -> checkpointLocation)) + class StateStoreKeyRowFormatValidationFailure(errorMsg: String) extends SparkRuntimeException( errorClass = "STATE_STORE_KEY_ROW_FORMAT_VALIDATION_FAILURE", @@ -264,3 +296,8 @@ class StateStoreValueRowFormatValidationFailure(errorMsg: String) extends SparkRuntimeException( errorClass = "STATE_STORE_VALUE_ROW_FORMAT_VALIDATION_FAILURE", messageParameters = Map("errorMsg" -> errorMsg)) + +class StateStoreProviderDoesNotSupportFineGrainedReplay(inputClass: String) + extends SparkUnsupportedOperationException( + errorClass = "STATE_STORE_PROVIDER_DOES_NOT_SUPPORT_FINE_GRAINED_STATE_REPLAY", + messageParameters = Map("inputClass" -> inputClass)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala index 43ffdf9829b4a..3957a24d5a8e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala @@ -87,7 +87,8 @@ class SymmetricHashJoinStateManager( partitionId: Int, stateFormatVersion: Int, skippedNullValueCount: Option[SQLMetric] = None, - useStateStoreCoordinator: Boolean = true) extends Logging { + useStateStoreCoordinator: Boolean = true, + snapshotStartVersion: Option[Long] = None) extends Logging { import SymmetricHashJoinStateManager._ /* @@ -480,6 +481,8 @@ class SymmetricHashJoinStateManager( val storeProviderId = StateStoreProviderId( stateInfo.get, partitionId, getStateStoreName(joinSide, stateStoreType)) val store = if (useStateStoreCoordinator) { + assert(snapshotStartVersion.isEmpty, "Should not use state store coordinator " + + "when reading state as data source.") StateStore.get( storeProviderId, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), stateInfo.get.storeVersion, useColumnFamilies = false, storeConf, hadoopConf) @@ -489,7 +492,16 @@ class SymmetricHashJoinStateManager( storeProviderId, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), useColumnFamilies = false, storeConf, hadoopConf, useMultipleValuesPerKey = false) - stateStoreProvider.getStore(stateInfo.get.storeVersion) + if (snapshotStartVersion.isDefined) { + if (!stateStoreProvider.isInstanceOf[SupportsFineGrainedReplay]) { + throw StateStoreErrors.stateStoreProviderDoesNotSupportFineGrainedReplay( + stateStoreProvider.getClass.toString) + } + stateStoreProvider.asInstanceOf[SupportsFineGrainedReplay] + .replayStateFromSnapshot(snapshotStartVersion.get, stateInfo.get.storeVersion) + } else { + stateStoreProvider.getStore(stateInfo.get.storeVersion) + } } logInfo(log"Loaded store ${MDC(STATE_STORE_ID, store.id)}") store diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/metadata new file mode 100644 index 0000000000000..6fb99c5969bd9 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/metadata @@ -0,0 +1 @@ +{"id":"ab7bcd9a-4146-45d3-933d-a615b381c3be"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/0 new file mode 100644 index 0000000000000..ba13971f3848b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290655102,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/1 new file mode 100644 index 0000000000000..5fd58f6716944 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290659041,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/2 new file mode 100644 index 0000000000000..be839a1efa191 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290661716,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/3 new file mode 100644 index 0000000000000..c87f4b1b97bcc --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290664278,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..20b45317e0a22 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/1.delta new file mode 100644 index 0000000000000..dae5bfc800597 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.delta new file mode 100644 index 0000000000000..4405af3420786 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.snapshot new file mode 100644 index 0000000000000..4405af3420786 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/3.delta new file mode 100644 index 0000000000000..433925dab114d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.delta new file mode 100644 index 0000000000000..064533dcd4f30 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.snapshot new file mode 100644 index 0000000000000..b4e23cc0ed4e5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/1/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.delta new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.snapshot new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/3.delta new file mode 100644 index 0000000000000..acf7619c291f3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.delta new file mode 100644 index 0000000000000..27ca4dcede3dc Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.snapshot new file mode 100644 index 0000000000000..27ca4dcede3dc Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/2/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/1.delta new file mode 100644 index 0000000000000..859c2b1315a5e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.snapshot new file mode 100644 index 0000000000000..859c2b1315a5e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.snapshot new file mode 100644 index 0000000000000..859c2b1315a5e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/3/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/1.delta new file mode 100644 index 0000000000000..0bdaf341003b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.delta new file mode 100644 index 0000000000000..3465b025dfa03 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.snapshot new file mode 100644 index 0000000000000..3465b025dfa03 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.delta new file mode 100644 index 0000000000000..6070683f44e12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.snapshot new file mode 100644 index 0000000000000..bf46f06c500d8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/4/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/_metadata/metadata new file mode 100644 index 0000000000000..5094b71aa3581 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/agg/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"stateStoreSave"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/metadata new file mode 100644 index 0000000000000..b151b6c27e031 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/metadata @@ -0,0 +1 @@ +{"id":"9253a36d-8c80-4cdc-9d2a-3cd9b5ceff59"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/0 new file mode 100644 index 0000000000000..acbebd05ef160 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289915309,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/1 new file mode 100644 index 0000000000000..3485e7b5927c1 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289918096,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/2 new file mode 100644 index 0000000000000..a5be1f2e758aa --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289921002,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/3 new file mode 100644 index 0000000000000..6a238cc687bf7 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289923707,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..70ebb77e1b781 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/1.delta new file mode 100644 index 0000000000000..aac301da2c2ab Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.snapshot new file mode 100644 index 0000000000000..aac301da2c2ab Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/3.delta new file mode 100644 index 0000000000000..c88091a5bbc9e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.snapshot new file mode 100644 index 0000000000000..8cd4b5dcb1ec2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/1/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.delta new file mode 100644 index 0000000000000..52dacd1351c7e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.snapshot new file mode 100644 index 0000000000000..52dacd1351c7e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.snapshot new file mode 100644 index 0000000000000..52dacd1351c7e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/2/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/1.delta new file mode 100644 index 0000000000000..d86baf6d41aa2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.snapshot new file mode 100644 index 0000000000000..d86baf6d41aa2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.snapshot new file mode 100644 index 0000000000000..d86baf6d41aa2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/3/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/1.delta new file mode 100644 index 0000000000000..1a985dfde9d45 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.snapshot new file mode 100644 index 0000000000000..1a985dfde9d45 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.delta new file mode 100644 index 0000000000000..9d22a051ebfdc Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.snapshot new file mode 100644 index 0000000000000..eb35f4815bfe8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/4/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/_metadata/metadata new file mode 100644 index 0000000000000..39ce28c9b4aa5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/dedup/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"dedupe"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/1 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/2 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/3 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/4 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/commits/4 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/metadata new file mode 100644 index 0000000000000..c3656bc51c886 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/metadata @@ -0,0 +1 @@ +{"id":"fbcd445d-b716-4589-ad15-774afa5a243d"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/0 new file mode 100644 index 0000000000000..4c4f47f8e9ec5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/0 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289926768,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/1 new file mode 100644 index 0000000000000..13cd6f3002a74 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/1 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289932001,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/2 new file mode 100644 index 0000000000000..013735ad0d044 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/2 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719289935688,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/3 new file mode 100644 index 0000000000000..6802d77f1ecc8 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/3 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719289939702,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/4 new file mode 100644 index 0000000000000..8621a231056b2 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/offsets/4 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":9000,"batchTimestampMs":1719289942516,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..42448b3b584ce Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..8fa8f1675bc82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/1/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..d295efee0d000 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..4460eadea0c0d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..7e6dce9cc108c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..21e2a706e8c7a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..5bfdddd0e2f14 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..30e88f6cb35c6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..d295efee0d000 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..4d3cf654ce551 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..2ec494e6a636f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..0877f7564366f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..5bfdddd0e2f14 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..edcc5dd1f672a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..0877f7564366f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/2/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..1aeeffd771c08 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..379479c0ccc3b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..cafd9d540f8e0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..69437319e872d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..aa76ff9b416a1 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..1aeeffd771c08 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..379479c0ccc3b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..47dff164d42d0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..105c7cc4255a9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..aa76ff9b416a1 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..d536c6fdcbce4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/3/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..0bdaf341003b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6070683f44e12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..837a1434917ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..32506a0366066 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..cf43298608153 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..2f2da77129ea4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..56b52ab974a3d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..26d97da9c610f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..2dbdd331b3e97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..3603b3f81bc77 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..b92431e0a4df6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..71fdc6c434ca3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..0bdaf341003b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6070683f44e12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..837a1434917ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..2f2da77129ea4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..32506a0366066 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..3b529f86101ac Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..83a5f723a34ab Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..0ed4feb1bd9b6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..31a686912dc97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..2dbdd331b3e97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..71fdc6c434ca3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..649772d35ffd8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..50b057756915b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..1c4c3974de1d5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/4/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/_metadata/metadata new file mode 100644 index 0000000000000..b73f1e3e66ac5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join1/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"symmetricHashJoin"},"stateStoreInfo":[{"storeName":"left-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"left-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/1 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/2 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/3 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/4 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/commits/4 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/metadata new file mode 100644 index 0000000000000..331d37d197a17 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/metadata @@ -0,0 +1 @@ +{"id":"26806544-709f-4745-a6e4-7641361fe94a"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/0 new file mode 100644 index 0000000000000..20e0fddaf646b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/0 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289946613,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/1 new file mode 100644 index 0000000000000..f71629438b9e2 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/1 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289951327,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/2 new file mode 100644 index 0000000000000..789561fd971a4 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/2 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719289954259,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/3 new file mode 100644 index 0000000000000..0d224bfc5920a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/3 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719289958068,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/4 new file mode 100644 index 0000000000000..68158e5f2542e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/offsets/4 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":9000,"batchTimestampMs":1719289960765,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..42448b3b584ce Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..8fa8f1675bc82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/1/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..d295efee0d000 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..4460eadea0c0d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..7e6dce9cc108c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..21e2a706e8c7a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..5bfdddd0e2f14 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..30e88f6cb35c6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..4e421cd377fb6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..d295efee0d000 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..4d3cf654ce551 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..25d24f4cedf3f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..2ec494e6a636f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..0877f7564366f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..5bfdddd0e2f14 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..edcc5dd1f672a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..0877f7564366f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/2/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..1aeeffd771c08 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..379479c0ccc3b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..cafd9d540f8e0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..69437319e872d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..aa76ff9b416a1 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..74fa1fc58b611 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..1aeeffd771c08 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..379479c0ccc3b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..47dff164d42d0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..9899fb58eebe4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..105c7cc4255a9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..aa76ff9b416a1 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..d536c6fdcbce4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/3/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/1.delta new file mode 100644 index 0000000000000..0bdaf341003b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6070683f44e12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..837a1434917ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.delta new file mode 100644 index 0000000000000..32506a0366066 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..cf43298608153 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/5.delta new file mode 100644 index 0000000000000..2f2da77129ea4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..56b52ab974a3d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..26d97da9c610f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..2dbdd331b3e97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..3603b3f81bc77 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..b92431e0a4df6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..71fdc6c434ca3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/left-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/1.delta new file mode 100644 index 0000000000000..0bdaf341003b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.delta new file mode 100644 index 0000000000000..6070683f44e12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.snapshot new file mode 100644 index 0000000000000..837a1434917ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/3.delta new file mode 100644 index 0000000000000..2f2da77129ea4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.delta new file mode 100644 index 0000000000000..32506a0366066 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.snapshot new file mode 100644 index 0000000000000..3b529f86101ac Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/5.delta new file mode 100644 index 0000000000000..83a5f723a34ab Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyToNumValues/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/1.delta new file mode 100644 index 0000000000000..0ed4feb1bd9b6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.delta new file mode 100644 index 0000000000000..31a686912dc97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.snapshot new file mode 100644 index 0000000000000..2dbdd331b3e97 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/3.delta new file mode 100644 index 0000000000000..71fdc6c434ca3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.delta new file mode 100644 index 0000000000000..649772d35ffd8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.snapshot new file mode 100644 index 0000000000000..50b057756915b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/5.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/5.delta new file mode 100644 index 0000000000000..1c4c3974de1d5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/4/right-keyWithIndexToValue/5.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/_metadata/metadata new file mode 100644 index 0000000000000..b73f1e3e66ac5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/join2/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"symmetricHashJoin"},"stateStoreInfo":[{"storeName":"left-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"left-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/metadata new file mode 100644 index 0000000000000..1d512939c8dca --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/metadata @@ -0,0 +1 @@ +{"id":"5ed656f0-84dd-414c-abbc-851e4ce58b93"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/0 new file mode 100644 index 0000000000000..95444ab5cb96a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289904957,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/1 new file mode 100644 index 0000000000000..614f4bb85c018 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289907608,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/2 new file mode 100644 index 0000000000000..b4f3aebbbe875 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289910090,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/3 new file mode 100644 index 0000000000000..7692f0f1abead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719289912575,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/1.delta new file mode 100644 index 0000000000000..701f5bd986b86 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.delta new file mode 100644 index 0000000000000..1de87347fb513 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.snapshot new file mode 100644 index 0000000000000..1de87347fb513 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/2.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/3.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/3.delta new file mode 100644 index 0000000000000..5b1d0ce87f287 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/3.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.delta new file mode 100644 index 0000000000000..cc309967b185e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.snapshot b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.snapshot new file mode 100644 index 0000000000000..cc309967b185e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/4.snapshot differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..371b0df09d80b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/_metadata/metadata new file mode 100644 index 0000000000000..5792421dd423e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/hdfs/limit/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"globalLimit"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/metadata new file mode 100644 index 0000000000000..24c81c5d47be5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/metadata @@ -0,0 +1 @@ +{"id":"0da797f0-6c12-4954-89cf-cea200a87f97"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/0 new file mode 100644 index 0000000000000..add7a6458926f --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290931555,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/1 new file mode 100644 index 0000000000000..2857388877a31 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290934440,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/2 new file mode 100644 index 0000000000000..5df970a41a729 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290937697,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/3 new file mode 100644 index 0000000000000..a8efebc1cf7ec --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290940519,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.zip new file mode 100644 index 0000000000000..b52a3e04cde93 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.zip new file mode 100644 index 0000000000000..97e703840ba71 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..20b45317e0a22 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/1.changelog new file mode 100644 index 0000000000000..497e792645825 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.changelog new file mode 100644 index 0000000000000..c197734e2608c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.zip new file mode 100644 index 0000000000000..4002b00264aac Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/3.changelog new file mode 100644 index 0000000000000..35985490c08d1 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.changelog new file mode 100644 index 0000000000000..bbd5a488fb858 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.zip new file mode 100644 index 0000000000000..00e9f17f64f12 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000008-e414df11-1a90-4be8-807c-8d6f970b5f56.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000008-e414df11-1a90-4be8-807c-8d6f970b5f56.sst new file mode 100644 index 0000000000000..c41d3cc2bafb8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000008-e414df11-1a90-4be8-807c-8d6f970b5f56.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000015-97d0e283-eed7-4af0-87d9-eded3afe15ca.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000015-97d0e283-eed7-4af0-87d9-eded3afe15ca.sst new file mode 100644 index 0000000000000..186fba2d13a5c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/1/SSTs/000015-97d0e283-eed7-4af0-87d9-eded3afe15ca.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.changelog new file mode 100644 index 0000000000000..2daeae74a72b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.zip new file mode 100644 index 0000000000000..1c1ae3f894804 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/3.changelog new file mode 100644 index 0000000000000..5d9eea4ad997e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.changelog new file mode 100644 index 0000000000000..1ddb6499c28c4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.zip new file mode 100644 index 0000000000000..e574f5e25473b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000008-d6862a4a-f10f-4641-ba89-0ea25cd816b0.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000008-d6862a4a-f10f-4641-ba89-0ea25cd816b0.sst new file mode 100644 index 0000000000000..9e38658bda2de Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000008-d6862a4a-f10f-4641-ba89-0ea25cd816b0.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000015-1e8225d7-793e-47df-aec6-fb47522e2494.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000015-1e8225d7-793e-47df-aec6-fb47522e2494.sst new file mode 100644 index 0000000000000..8f4391b954d05 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/2/SSTs/000015-1e8225d7-793e-47df-aec6-fb47522e2494.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/1.changelog new file mode 100644 index 0000000000000..46e739559ac8d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.zip new file mode 100644 index 0000000000000..847766f4eb425 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.zip new file mode 100644 index 0000000000000..a2b18fa0d827e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/SSTs/000008-637e442e-feb6-4022-a0b1-919d388073ae.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/SSTs/000008-637e442e-feb6-4022-a0b1-919d388073ae.sst new file mode 100644 index 0000000000000..66dd9dfb28c19 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/3/SSTs/000008-637e442e-feb6-4022-a0b1-919d388073ae.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/1.changelog new file mode 100644 index 0000000000000..9099712c6134d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.changelog new file mode 100644 index 0000000000000..1152f1a527f20 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.zip new file mode 100644 index 0000000000000..9a806e422149f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.changelog new file mode 100644 index 0000000000000..c5a3af6244a01 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.zip new file mode 100644 index 0000000000000..eebcf56f8533c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000008-0f086ebd-c188-4508-97d3-9282313b0a97.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000008-0f086ebd-c188-4508-97d3-9282313b0a97.sst new file mode 100644 index 0000000000000..fcf1c915751ea Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000008-0f086ebd-c188-4508-97d3-9282313b0a97.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000015-074aa977-f8df-4b58-8143-c5c4691cdb9a.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000015-074aa977-f8df-4b58-8143-c5c4691cdb9a.sst new file mode 100644 index 0000000000000..7522d3bd7cadb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/4/SSTs/000015-074aa977-f8df-4b58-8143-c5c4691cdb9a.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/_metadata/metadata new file mode 100644 index 0000000000000..5094b71aa3581 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/agg/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"stateStoreSave"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/metadata new file mode 100644 index 0000000000000..46c70457ae13f --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/metadata @@ -0,0 +1 @@ +{"id":"7715e3b8-26dd-416a-ac64-c804bb48a972"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/0 new file mode 100644 index 0000000000000..ef98649c9fc3a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290943839,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/1 new file mode 100644 index 0000000000000..46d8177c1ed43 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290946665,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/2 new file mode 100644 index 0000000000000..dc7562efc9fd2 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290949666,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/3 new file mode 100644 index 0000000000000..52f25fb6acccd --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290952214,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.zip new file mode 100644 index 0000000000000..5910abeed315f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.zip new file mode 100644 index 0000000000000..ddba4b5772e50 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..70ebb77e1b781 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/1.changelog new file mode 100644 index 0000000000000..1132252c8945f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.zip new file mode 100644 index 0000000000000..0bf7ee845bc08 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/3.changelog new file mode 100644 index 0000000000000..c5ebbc7bbed26 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.zip new file mode 100644 index 0000000000000..33af823d62d1f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000008-3073b336-661b-4d30-984d-427402150d37.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000008-3073b336-661b-4d30-984d-427402150d37.sst new file mode 100644 index 0000000000000..20f8285636f8f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000008-3073b336-661b-4d30-984d-427402150d37.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000009-5669c494-2412-46c4-96ef-1bb9ab6b2710.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000009-5669c494-2412-46c4-96ef-1bb9ab6b2710.sst new file mode 100644 index 0000000000000..cf9e952972a54 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/1/SSTs/000009-5669c494-2412-46c4-96ef-1bb9ab6b2710.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.changelog new file mode 100644 index 0000000000000..f96b5ad48b3f8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.zip new file mode 100644 index 0000000000000..bc12dd5ba4c2d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.zip new file mode 100644 index 0000000000000..9a720d8720d88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/SSTs/000008-90e6688f-83a0-4917-ae32-9e107c002bcc.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/SSTs/000008-90e6688f-83a0-4917-ae32-9e107c002bcc.sst new file mode 100644 index 0000000000000..3bbb3466e6f82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/2/SSTs/000008-90e6688f-83a0-4917-ae32-9e107c002bcc.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/1.changelog new file mode 100644 index 0000000000000..b3f54eeacdd04 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.zip new file mode 100644 index 0000000000000..ae313425f0644 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.zip new file mode 100644 index 0000000000000..fa589eca47e36 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/SSTs/000008-e115799d-89cb-4f4d-8e58-fe104d382c80.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/SSTs/000008-e115799d-89cb-4f4d-8e58-fe104d382c80.sst new file mode 100644 index 0000000000000..8b786dd457f96 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/3/SSTs/000008-e115799d-89cb-4f4d-8e58-fe104d382c80.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/1.changelog new file mode 100644 index 0000000000000..9b224728c8bcb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.zip new file mode 100644 index 0000000000000..faf5ea1b8158b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.changelog new file mode 100644 index 0000000000000..6fc1bb07e798d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.zip new file mode 100644 index 0000000000000..03837a7357d71 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000008-5f249adf-c98e-46a3-8217-e39a4360b624.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000008-5f249adf-c98e-46a3-8217-e39a4360b624.sst new file mode 100644 index 0000000000000..a17404703c22a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000008-5f249adf-c98e-46a3-8217-e39a4360b624.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000009-0d1befcf-847e-43a5-b354-a4d11afeaaba.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000009-0d1befcf-847e-43a5-b354-a4d11afeaaba.sst new file mode 100644 index 0000000000000..5d68995c5c8b0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/4/SSTs/000009-0d1befcf-847e-43a5-b354-a4d11afeaaba.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/_metadata/metadata new file mode 100644 index 0000000000000..39ce28c9b4aa5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/dedup/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"dedupe"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/1 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/2 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/3 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/4 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/commits/4 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/metadata new file mode 100644 index 0000000000000..e0a261d1ce0e2 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/metadata @@ -0,0 +1 @@ +{"id":"160224de-518b-4191-b9d6-544602476824"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/0 new file mode 100644 index 0000000000000..67dd9ed17d96c --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/0 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290955223,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/1 new file mode 100644 index 0000000000000..dd1850eb28417 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/1 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290960126,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/2 new file mode 100644 index 0000000000000..65fe14e65bd8d --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/2 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719290962748,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/3 new file mode 100644 index 0000000000000..756be72687efd --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/3 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719290966185,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/4 new file mode 100644 index 0000000000000..d2cbfe3c08429 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/offsets/4 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":9000,"batchTimestampMs":1719290968718,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..42448b3b584ce Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/left-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..8fa8f1675bc82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/0/right-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..0bedf21fe2da4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..59a6ea16dc12b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/1/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..2daeae74a72b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..abdd6a5fca257 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..3802a7e7b1ee8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..f5d38ea63e7b6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/SSTs/000008-97286fb3-8d19-4e91-aecb-3f9647f6103f.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/SSTs/000008-97286fb3-8d19-4e91-aecb-3f9647f6103f.sst new file mode 100644 index 0000000000000..d7494daaea026 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyToNumValues/SSTs/000008-97286fb3-8d19-4e91-aecb-3f9647f6103f.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..88a2191cae130 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..57d0436d95d79 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..1d20ea2ddd926 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..171c10c2f069c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/SSTs/000008-44099692-68de-4b22-8238-3a8029f693ad.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/SSTs/000008-44099692-68de-4b22-8238-3a8029f693ad.sst new file mode 100644 index 0000000000000..865f1fef7f1f5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/left-keyWithIndexToValue/SSTs/000008-44099692-68de-4b22-8238-3a8029f693ad.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..2daeae74a72b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..abdd6a5fca257 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..c3a661fc13ade Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..e8da61cb5ddc9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..f6157326c212d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000008-a9a5b2ba-3bff-402f-b750-b9232ac3d1b1.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000008-a9a5b2ba-3bff-402f-b750-b9232ac3d1b1.sst new file mode 100644 index 0000000000000..a9160d9a66ab6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000008-a9a5b2ba-3bff-402f-b750-b9232ac3d1b1.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000009-c5863121-41a9-4ffc-9c54-5f4c2a1139a2.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000009-c5863121-41a9-4ffc-9c54-5f4c2a1139a2.sst new file mode 100644 index 0000000000000..f4987ae824e82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyToNumValues/SSTs/000009-c5863121-41a9-4ffc-9c54-5f4c2a1139a2.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..14f1066107657 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..e1452512bade8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..6411680f0c8fb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6b20acbc9b79b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..982f220c2cbd7 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000008-a55c633d-a594-4cac-b6f2-c63907cb9581.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000008-a55c633d-a594-4cac-b6f2-c63907cb9581.sst new file mode 100644 index 0000000000000..6464d1dbfcc7e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000008-a55c633d-a594-4cac-b6f2-c63907cb9581.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000009-f8390e48-d464-49b8-8a3b-6393448f907d.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000009-f8390e48-d464-49b8-8a3b-6393448f907d.sst new file mode 100644 index 0000000000000..df6034ac2f3db Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/2/right-keyWithIndexToValue/SSTs/000009-f8390e48-d464-49b8-8a3b-6393448f907d.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..99210c3a3c567 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..d02b01bc35f23 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..262abc1cc08a5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..e198b32fff211 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000008-ad09ce4b-4be4-44ce-a620-4244a73f84f5.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000008-ad09ce4b-4be4-44ce-a620-4244a73f84f5.sst new file mode 100644 index 0000000000000..07de5247476c7 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000008-ad09ce4b-4be4-44ce-a620-4244a73f84f5.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000009-5561a93a-1e0b-46aa-98d5-685048b992fa.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000009-5561a93a-1e0b-46aa-98d5-685048b992fa.sst new file mode 100644 index 0000000000000..7619f71cd55ad Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyToNumValues/SSTs/000009-5561a93a-1e0b-46aa-98d5-685048b992fa.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..150d775115a47 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..91cea46a6cd86 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..3964e93f5a400 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..9319b07246660 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000008-36205897-8ada-4516-b663-122c915754f0.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000008-36205897-8ada-4516-b663-122c915754f0.sst new file mode 100644 index 0000000000000..cad982214ad18 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000008-36205897-8ada-4516-b663-122c915754f0.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000009-bffa46f8-b6c7-431e-a579-d8725aec338b.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000009-bffa46f8-b6c7-431e-a579-d8725aec338b.sst new file mode 100644 index 0000000000000..aa392eaadd1d9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/left-keyWithIndexToValue/SSTs/000009-bffa46f8-b6c7-431e-a579-d8725aec338b.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..99210c3a3c567 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..3ae503e5deff3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..262abc1cc08a5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..4ba14fb5af442 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..5074a90bc3b90 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000008-f6d87143-b4d6-4c30-bd43-8e90c98704e9.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000008-f6d87143-b4d6-4c30-bd43-8e90c98704e9.sst new file mode 100644 index 0000000000000..e3dbd832a2321 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000008-f6d87143-b4d6-4c30-bd43-8e90c98704e9.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000009-64a8ab01-cdfe-4c82-94e3-5bc4c64b6671.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000009-64a8ab01-cdfe-4c82-94e3-5bc4c64b6671.sst new file mode 100644 index 0000000000000..4801a5b8254f7 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyToNumValues/SSTs/000009-64a8ab01-cdfe-4c82-94e3-5bc4c64b6671.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6d01a6e79416f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..6a9f94a8b33bf Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..28785bebe081e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..5b8f0d2bbf151 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..ff933a6cfd56c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000008-50764781-b9d2-482d-a307-b2d1cab8c639.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000008-50764781-b9d2-482d-a307-b2d1cab8c639.sst new file mode 100644 index 0000000000000..9a627936b57ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000008-50764781-b9d2-482d-a307-b2d1cab8c639.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000009-38fca8e3-893a-4d8b-8f4a-57366102a54c.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000009-38fca8e3-893a-4d8b-8f4a-57366102a54c.sst new file mode 100644 index 0000000000000..307b166f4e9b3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/3/right-keyWithIndexToValue/SSTs/000009-38fca8e3-893a-4d8b-8f4a-57366102a54c.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..9099712c6134d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..c5a3af6244a01 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..1ac5a3348bbe6 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..4b3cfd1f719bb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..54e0959698acf Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..298ebddcda718 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000008-fd11870f-bea5-4721-aedf-b14399b966bd.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000008-fd11870f-bea5-4721-aedf-b14399b966bd.sst new file mode 100644 index 0000000000000..4ae4e252afb2a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000008-fd11870f-bea5-4721-aedf-b14399b966bd.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000009-1ba9d77a-6c31-42e5-a18e-f66c0ef201b5.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000009-1ba9d77a-6c31-42e5-a18e-f66c0ef201b5.sst new file mode 100644 index 0000000000000..c3dd64fbe2805 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyToNumValues/SSTs/000009-1ba9d77a-6c31-42e5-a18e-f66c0ef201b5.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..a7735153b54cc Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..f4407b5c4f83b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..ee8a442d7eaa0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..3e2ea96d857f0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..2dfce06a4435b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..9ec7dcb0d0515 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000008-7fbc8ecb-8e90-4de6-a334-ce789dfe3dd5.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000008-7fbc8ecb-8e90-4de6-a334-ce789dfe3dd5.sst new file mode 100644 index 0000000000000..6b5b6c05ae0d2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000008-7fbc8ecb-8e90-4de6-a334-ce789dfe3dd5.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000009-05c7d657-5d52-4d9d-9e5e-d28daf8d6500.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000009-05c7d657-5d52-4d9d-9e5e-d28daf8d6500.sst new file mode 100644 index 0000000000000..72187124c13f2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/left-keyWithIndexToValue/SSTs/000009-05c7d657-5d52-4d9d-9e5e-d28daf8d6500.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..9099712c6134d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..c5a3af6244a01 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..ed87d10aecafa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..298ebddcda718 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..4b3cfd1f719bb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..b777691d26852 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..d50df0481f970 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000008-fb249fa1-17cf-4f28-99fa-e31932ec1caf.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000008-fb249fa1-17cf-4f28-99fa-e31932ec1caf.sst new file mode 100644 index 0000000000000..c8b94f7b4bc85 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000008-fb249fa1-17cf-4f28-99fa-e31932ec1caf.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000009-6dc46235-1157-4bb7-9af0-58aee4e2edf2.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000009-6dc46235-1157-4bb7-9af0-58aee4e2edf2.sst new file mode 100644 index 0000000000000..564acccff46f8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyToNumValues/SSTs/000009-6dc46235-1157-4bb7-9af0-58aee4e2edf2.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..32458caceea0a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..a3bc5bfd26f8e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..54fd13c914ae4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..9ec7dcb0d0515 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..e3906cea3366a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..97ba13e356c67 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..ffe4b92ca56dd Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000008-2695fe50-483f-4beb-a12d-770bfe63da9c.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000008-2695fe50-483f-4beb-a12d-770bfe63da9c.sst new file mode 100644 index 0000000000000..fa6f037ecb6ab Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000008-2695fe50-483f-4beb-a12d-770bfe63da9c.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000009-c819e3a7-826b-46d0-8667-1670b9b2d13c.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000009-c819e3a7-826b-46d0-8667-1670b9b2d13c.sst new file mode 100644 index 0000000000000..9db30172f1477 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/4/right-keyWithIndexToValue/SSTs/000009-c819e3a7-826b-46d0-8667-1670b9b2d13c.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/_metadata/metadata new file mode 100644 index 0000000000000..b73f1e3e66ac5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join1/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"symmetricHashJoin"},"stateStoreInfo":[{"storeName":"left-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"left-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/1 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/2 new file mode 100644 index 0000000000000..1715390973f9b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":5000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/3 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/4 new file mode 100644 index 0000000000000..2a2d02b0bb3b3 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/commits/4 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":9000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/metadata new file mode 100644 index 0000000000000..67f735ddd54b5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/metadata @@ -0,0 +1 @@ +{"id":"cb2f2800-2129-4356-bf83-e985db7d8556"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/0 new file mode 100644 index 0000000000000..fe391df89fb42 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/0 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290972508,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/1 new file mode 100644 index 0000000000000..448160d0ae047 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/1 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290977372,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/2 new file mode 100644 index 0000000000000..740bc6b031054 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/2 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719290979966,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/3 new file mode 100644 index 0000000000000..1987fabf89c19 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/3 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":5000,"batchTimestampMs":1719290983375,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/4 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/4 new file mode 100644 index 0000000000000..7eeef49c1f168 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/offsets/4 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":9000,"batchTimestampMs":1719290985871,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..89a73981724aa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..85424f0d5afba Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..89a73981724aa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..85424f0d5afba Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..42448b3b584ce Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/left-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..993d32e965db5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..cfcea9ae02e88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/_metadata/schema new file mode 100644 index 0000000000000..4da637d143496 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyToNumValues/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..993d32e965db5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..cfcea9ae02e88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema new file mode 100644 index 0000000000000..8fa8f1675bc82 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/0/right-keyWithIndexToValue/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..89a73981724aa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..cfcea9ae02e88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..89a73981724aa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..85424f0d5afba Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..993d32e965db5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..cfcea9ae02e88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..993d32e965db5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..cfcea9ae02e88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/1/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..2daeae74a72b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..abdd6a5fca257 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..23e8882b8d9f2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..218dc89456c8b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/SSTs/000008-bfd9de3b-a339-42ad-8270-343b70b3999e.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/SSTs/000008-bfd9de3b-a339-42ad-8270-343b70b3999e.sst new file mode 100644 index 0000000000000..6dcf6ef780c87 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyToNumValues/SSTs/000008-bfd9de3b-a339-42ad-8270-343b70b3999e.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..88a2191cae130 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..57d0436d95d79 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..46803957b004a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..2d25b2447e398 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/SSTs/000008-daf5150d-98b0-4042-ac23-f6ffcb9eef2f.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/SSTs/000008-daf5150d-98b0-4042-ac23-f6ffcb9eef2f.sst new file mode 100644 index 0000000000000..10fd7fc691c70 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/left-keyWithIndexToValue/SSTs/000008-daf5150d-98b0-4042-ac23-f6ffcb9eef2f.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..2daeae74a72b9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..abdd6a5fca257 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..ad3d69d2a8789 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..e8da61cb5ddc9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..c89c97f5641fe Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000008-8de57e57-02b1-47fc-8150-eb984771e1ca.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000008-8de57e57-02b1-47fc-8150-eb984771e1ca.sst new file mode 100644 index 0000000000000..4744bbf75039b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000008-8de57e57-02b1-47fc-8150-eb984771e1ca.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000009-a8eb1b10-f5fe-4a44-94ab-545d1e33121c.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000009-a8eb1b10-f5fe-4a44-94ab-545d1e33121c.sst new file mode 100644 index 0000000000000..5706bad6347fe Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyToNumValues/SSTs/000009-a8eb1b10-f5fe-4a44-94ab-545d1e33121c.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..14f1066107657 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..e1452512bade8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..a600351f2674f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6b20acbc9b79b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..4d53649b5608c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000008-4f4617f6-01ce-48f5-aaa9-c9d7007ac482.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000008-4f4617f6-01ce-48f5-aaa9-c9d7007ac482.sst new file mode 100644 index 0000000000000..650af4d3042de Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000008-4f4617f6-01ce-48f5-aaa9-c9d7007ac482.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000009-840bae7d-ff30-4aef-9672-bf70fa6943c4.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000009-840bae7d-ff30-4aef-9672-bf70fa6943c4.sst new file mode 100644 index 0000000000000..e1ed48d909d9c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/2/right-keyWithIndexToValue/SSTs/000009-840bae7d-ff30-4aef-9672-bf70fa6943c4.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..99210c3a3c567 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..2123cc3a99531 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..262abc1cc08a5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..83efaf04ab162 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000008-e1bd2941-d044-4898-a841-7d6d094b7c30.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000008-e1bd2941-d044-4898-a841-7d6d094b7c30.sst new file mode 100644 index 0000000000000..37af8fb0bf6b4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000008-e1bd2941-d044-4898-a841-7d6d094b7c30.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000009-e646f573-4a5b-408d-88da-85810aabe966.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000009-e646f573-4a5b-408d-88da-85810aabe966.sst new file mode 100644 index 0000000000000..149350b5abec4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyToNumValues/SSTs/000009-e646f573-4a5b-408d-88da-85810aabe966.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..150d775115a47 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..4f0ff11d17441 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..3964e93f5a400 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..c4ac694c0f31b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000008-28cb1ac6-cf15-4f63-8c74-ea655f8ce669.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000008-28cb1ac6-cf15-4f63-8c74-ea655f8ce669.sst new file mode 100644 index 0000000000000..cbf2c438d8437 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000008-28cb1ac6-cf15-4f63-8c74-ea655f8ce669.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000009-5a4a38a3-8e5a-4436-b0b7-718fe7ed09a6.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000009-5a4a38a3-8e5a-4436-b0b7-718fe7ed09a6.sst new file mode 100644 index 0000000000000..f3769a666d813 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/left-keyWithIndexToValue/SSTs/000009-5a4a38a3-8e5a-4436-b0b7-718fe7ed09a6.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..99210c3a3c567 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..4a5a9bd672051 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..262abc1cc08a5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..52e86b7a99073 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..5074a90bc3b90 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000008-294ce63e-1e67-4bc5-98fa-67b6f40669ff.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000008-294ce63e-1e67-4bc5-98fa-67b6f40669ff.sst new file mode 100644 index 0000000000000..0e93b773ae5ca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000008-294ce63e-1e67-4bc5-98fa-67b6f40669ff.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000009-ad3aeaf9-d804-4237-91e5-e2d61f7c8338.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000009-ad3aeaf9-d804-4237-91e5-e2d61f7c8338.sst new file mode 100644 index 0000000000000..2eb1556d44bac Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyToNumValues/SSTs/000009-ad3aeaf9-d804-4237-91e5-e2d61f7c8338.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..6d01a6e79416f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..e64d4d046de88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..28785bebe081e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..e5fda4000120f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..ff933a6cfd56c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000008-e9c9dd20-a2b3-4eac-94e6-3336a8bf2953.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000008-e9c9dd20-a2b3-4eac-94e6-3336a8bf2953.sst new file mode 100644 index 0000000000000..a737c2036d013 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000008-e9c9dd20-a2b3-4eac-94e6-3336a8bf2953.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000009-7db59f7a-dabc-41f7-8878-87ca30afb09d.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000009-7db59f7a-dabc-41f7-8878-87ca30afb09d.sst new file mode 100644 index 0000000000000..4af5716c9c24e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/3/right-keyWithIndexToValue/SSTs/000009-7db59f7a-dabc-41f7-8878-87ca30afb09d.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..9099712c6134d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..c5a3af6244a01 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.zip new file mode 100644 index 0000000000000..a62d4d6504f88 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..4b3cfd1f719bb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.zip new file mode 100644 index 0000000000000..3003df2131bf3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..298ebddcda718 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000008-a9c68839-18e4-4ef3-a329-432d2810b0c3.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000008-a9c68839-18e4-4ef3-a329-432d2810b0c3.sst new file mode 100644 index 0000000000000..5b9827d63d689 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000008-a9c68839-18e4-4ef3-a329-432d2810b0c3.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000009-e91db969-62ec-45d7-a7da-a4301e129feb.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000009-e91db969-62ec-45d7-a7da-a4301e129feb.sst new file mode 100644 index 0000000000000..ad1f76fb3e750 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyToNumValues/SSTs/000009-e91db969-62ec-45d7-a7da-a4301e129feb.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..a7735153b54cc Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..f4407b5c4f83b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..b251d50f7e4ba Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..3e2ea96d857f0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..1873033aa1c9a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..9ec7dcb0d0515 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000008-5e19d13c-8855-47c9-8d4b-f14964f7fdfa.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000008-5e19d13c-8855-47c9-8d4b-f14964f7fdfa.sst new file mode 100644 index 0000000000000..71dec68b333a2 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000008-5e19d13c-8855-47c9-8d4b-f14964f7fdfa.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000009-a97d5e47-e4eb-4878-b62d-30cc77dac3b1.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000009-a97d5e47-e4eb-4878-b62d-30cc77dac3b1.sst new file mode 100644 index 0000000000000..a562e80e27871 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/left-keyWithIndexToValue/SSTs/000009-a97d5e47-e4eb-4878-b62d-30cc77dac3b1.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/1.changelog new file mode 100644 index 0000000000000..9099712c6134d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.changelog new file mode 100644 index 0000000000000..c5a3af6244a01 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.zip new file mode 100644 index 0000000000000..5c32348084ebb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/3.changelog new file mode 100644 index 0000000000000..298ebddcda718 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.changelog new file mode 100644 index 0000000000000..4b3cfd1f719bb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.zip new file mode 100644 index 0000000000000..954265fe88cfa Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/5.changelog new file mode 100644 index 0000000000000..d50df0481f970 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000008-432ddd3b-981c-4a2b-a1e4-e06ecbc4a8a7.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000008-432ddd3b-981c-4a2b-a1e4-e06ecbc4a8a7.sst new file mode 100644 index 0000000000000..e752148099e3d Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000008-432ddd3b-981c-4a2b-a1e4-e06ecbc4a8a7.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000009-2e821d73-a2c6-4261-bcb6-d69e82e4c0d3.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000009-2e821d73-a2c6-4261-bcb6-d69e82e4c0d3.sst new file mode 100644 index 0000000000000..27f61ffe48bf8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyToNumValues/SSTs/000009-2e821d73-a2c6-4261-bcb6-d69e82e4c0d3.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/1.changelog new file mode 100644 index 0000000000000..32458caceea0a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.changelog new file mode 100644 index 0000000000000..a3bc5bfd26f8e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.zip new file mode 100644 index 0000000000000..13bd0a0979309 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/3.changelog new file mode 100644 index 0000000000000..9ec7dcb0d0515 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.changelog new file mode 100644 index 0000000000000..e3906cea3366a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.zip new file mode 100644 index 0000000000000..d403db9b3e121 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/5.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/5.changelog new file mode 100644 index 0000000000000..ffe4b92ca56dd Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/5.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000008-4e1c24f5-bbcb-4797-95d9-cfa4de1be09d.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000008-4e1c24f5-bbcb-4797-95d9-cfa4de1be09d.sst new file mode 100644 index 0000000000000..2aafcbeadee17 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000008-4e1c24f5-bbcb-4797-95d9-cfa4de1be09d.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000009-366ba158-6a59-4922-9cb0-df3e2b9aa789.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000009-366ba158-6a59-4922-9cb0-df3e2b9aa789.sst new file mode 100644 index 0000000000000..68f1e65fa2885 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/4/right-keyWithIndexToValue/SSTs/000009-366ba158-6a59-4922-9cb0-df3e2b9aa789.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/_metadata/metadata new file mode 100644 index 0000000000000..b73f1e3e66ac5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/join2/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"symmetricHashJoin"},"stateStoreInfo":[{"storeName":"left-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"left-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyToNumValues","numColsPrefixKey":0,"numPartitions":5},{"storeName":"right-keyWithIndexToValue","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/2 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/2 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/3 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/commits/3 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/metadata new file mode 100644 index 0000000000000..fa82647084453 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/metadata @@ -0,0 +1 @@ +{"id":"711e4377-ce90-456e-8530-9e0374ce4791"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/0 new file mode 100644 index 0000000000000..50bf84e3a759e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290921178,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/1 new file mode 100644 index 0000000000000..1690ffdd2b79b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290923799,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/2 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/2 new file mode 100644 index 0000000000000..c331cc80a3e1e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/2 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290926380,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/3 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/3 new file mode 100644 index 0000000000000..2d42b90630780 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/offsets/3 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1719290928784,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.statefulOperator.useStrictDistribution":"true","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4"}} +3 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/1.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/1.changelog new file mode 100644 index 0000000000000..1d9f25b472ace Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/1.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.changelog new file mode 100644 index 0000000000000..c335f52e07bca Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.zip new file mode 100644 index 0000000000000..e707c3f31376a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/2.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/3.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/3.changelog new file mode 100644 index 0000000000000..41e61d1915140 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/3.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.changelog b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.changelog new file mode 100644 index 0000000000000..d51b4fc081740 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.changelog differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.zip b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.zip new file mode 100644 index 0000000000000..35f294cf782ce Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/4.zip differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000008-8f6e7567-9c70-4f55-9f28-8b6b497c85ae.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000008-8f6e7567-9c70-4f55-9f28-8b6b497c85ae.sst new file mode 100644 index 0000000000000..e8da97359c7fb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000008-8f6e7567-9c70-4f55-9f28-8b6b497c85ae.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000009-71eca19b-ab68-4608-bcfa-897365067239.sst b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000009-71eca19b-ab68-4608-bcfa-897365067239.sst new file mode 100644 index 0000000000000..fd296636929b8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/SSTs/000009-71eca19b-ab68-4608-bcfa-897365067239.sst differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..371b0df09d80b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/_metadata/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/_metadata/metadata new file mode 100644 index 0000000000000..5792421dd423e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-4.0.0/rocksdb/limit/state/0/_metadata/metadata @@ -0,0 +1,2 @@ +v1 +{"operatorInfo":{"operatorId":0,"operatorName":"globalLimit"},"stateStoreInfo":[{"storeName":"default","numColsPrefixKey":0,"numPartitions":5}]} \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala index c800168b507a8..e6cdd0dce9efa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala @@ -18,16 +18,18 @@ package org.apache.spark.sql.execution.datasources.v2.state import java.io.{File, FileWriter} +import org.apache.hadoop.conf.Configuration import org.scalatest.Assertions -import org.apache.spark.SparkUnsupportedOperationException +import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row} import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog} -import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore} +import org.apache.spark.sql.execution.streaming.state._ +import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{IntegerType, StructType} @@ -194,6 +196,78 @@ class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase { } } } + + test("ERROR: snapshotStartBatchId specified as a negative value") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceInvalidOptionValueIsNegative] { + spark.read.format("statestore") + // trick to bypass getting the last committed batch before validating operator ID + .option(StateSourceOptions.BATCH_ID, 0) + .option(StateSourceOptions.SNAPSHOT_START_BATCH_ID, -1) + .load(tempDir.getAbsolutePath) + } + checkError(exc, "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE", "42616", + Map("optionName" -> StateSourceOptions.SNAPSHOT_START_BATCH_ID)) + } + } + + test("ERROR: snapshotPartitionId specified as a negative value") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceInvalidOptionValueIsNegative] { + spark.read.format("statestore") + // trick to bypass getting the last committed batch before validating operator ID + .option(StateSourceOptions.BATCH_ID, 0) + .option(StateSourceOptions.SNAPSHOT_PARTITION_ID, -1) + .load(tempDir.getAbsolutePath) + } + checkError(exc, "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE", "42616", + Map("optionName" -> StateSourceOptions.SNAPSHOT_PARTITION_ID)) + } + } + + test("ERROR: snapshotStartBatchId specified without snapshotPartitionId or vice versa") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceUnspecifiedRequiredOption] { + spark.read.format("statestore") + // trick to bypass getting the last committed batch before validating operator ID + .option(StateSourceOptions.BATCH_ID, 0) + .option(StateSourceOptions.SNAPSHOT_START_BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + } + checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601", + Map("optionName" -> StateSourceOptions.SNAPSHOT_PARTITION_ID)) + } + + withTempDir { tempDir => + val exc = intercept[StateDataSourceUnspecifiedRequiredOption] { + spark.read.format("statestore") + // trick to bypass getting the last committed batch before validating operator ID + .option(StateSourceOptions.BATCH_ID, 0) + .option(StateSourceOptions.SNAPSHOT_PARTITION_ID, 0) + .load(tempDir.getAbsolutePath) + } + checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601", + Map("optionName" -> StateSourceOptions.SNAPSHOT_START_BATCH_ID)) + } + } + + test("ERROR: snapshotStartBatchId is greater than snapshotEndBatchId") { + withTempDir { tempDir => + val startBatchId = 1 + val endBatchId = 0 + val exc = intercept[StateDataSourceInvalidOptionValue] { + spark.read.format("statestore") + // trick to bypass getting the last committed batch before validating operator ID + .option(StateSourceOptions.SNAPSHOT_START_BATCH_ID, startBatchId) + .option(StateSourceOptions.BATCH_ID, endBatchId) + .load(tempDir.getAbsolutePath) + } + checkError(exc, "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE", "42616", + Map( + "optionName" -> StateSourceOptions.SNAPSHOT_START_BATCH_ID, + "message" -> s"value should be less than or equal to $endBatchId")) + } + } } /** @@ -301,35 +375,138 @@ class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase { } class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite { + override protected def newStateStoreProvider(): HDFSBackedStateStoreProvider = + new HDFSBackedStateStoreProvider + override def beforeAll(): Unit = { super.beforeAll() spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, - classOf[HDFSBackedStateStoreProvider].getName) + newStateStoreProvider().getClass.getName) + // make sure we have a snapshot for every two delta files + // HDFS maintenance task will not count the latest delta file, which has the same version + // as the snapshot version + spark.conf.set(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key, 1) + } + + test("ERROR: snapshot of version not found") { + testSnapshotNotFound() + } + + test("provider.replayReadStoreFromSnapshot(snapshotVersion, endVersion)") { + testGetReadStoreWithStartVersion() + } + + test("option snapshotPartitionId") { + testSnapshotPartitionId() + } + + test("snapshotStatBatchId on limit state") { + testSnapshotOnLimitState("hdfs") + } + + test("snapshotStatBatchId on aggregation state") { + testSnapshotOnAggregateState("hdfs") + } + + test("snapshotStatBatchId on deduplication state") { + testSnapshotOnDeduplicateState("hdfs") + } + + test("snapshotStatBatchId on join state") { + testSnapshotOnJoinState("hdfs", 1) + testSnapshotOnJoinState("hdfs", 2) } } class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite { + override protected def newStateStoreProvider(): RocksDBStateStoreProvider = + new RocksDBStateStoreProvider + override def beforeAll(): Unit = { super.beforeAll() spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, - classOf[RocksDBStateStoreProvider].getName) + newStateStoreProvider().getClass.getName) spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled", "false") } } -class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends StateDataSourceReadSuite { +class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends +StateDataSourceReadSuite { + override protected def newStateStoreProvider(): RocksDBStateStoreProvider = + new RocksDBStateStoreProvider + override def beforeAll(): Unit = { super.beforeAll() spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, - classOf[RocksDBStateStoreProvider].getName) + newStateStoreProvider().getClass.getName) spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled", "true") + // make sure we have a snapshot for every other checkpoint + // RocksDB maintenance task will count the latest checkpoint, so we need to set it to 2 + spark.conf.set(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key, 2) + } + + test("ERROR: snapshot of version not found") { + testSnapshotNotFound() + } + + test("provider.getReadStore(snapshotVersion, endVersion)") { + testGetReadStoreWithStartVersion() + } + + test("option snapshotPartitionId") { + testSnapshotPartitionId() + } + + test("snapshotStatBatchId on limit state") { + testSnapshotOnLimitState("rocksdb") + } + + test("snapshotStatBatchId on aggregation state") { + testSnapshotOnAggregateState("rocksdb") + } + + test("snapshotStatBatchId on deduplication state") { + testSnapshotOnDeduplicateState("rocksdb") + } + + test("snapshotStatBatchId on join state") { + testSnapshotOnJoinState("rocksdb", 1) + testSnapshotOnJoinState("rocksdb", 2) } } abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Assertions { + import testImplicits._ + import StateStoreTestsHelper._ + + protected val keySchema: StructType = StateStoreTestsHelper.keySchema + protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema + + protected def newStateStoreProvider(): StateStoreProvider + + /** + * Calls the overridable [[newStateStoreProvider]] to create the state store provider instance. + * Initialize it with the configuration set by child classes. + * + * @param checkpointDir path to store state information + * @return instance of class extending [[StateStoreProvider]] + */ + private def getNewStateStoreProvider(checkpointDir: String): StateStoreProvider = { + val provider = newStateStoreProvider() + provider.init( + StateStoreId(checkpointDir, 0, 0), + keySchema, + valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema), + useColumnFamilies = false, + StateStoreConf(spark.sessionState.conf), + new Configuration) + provider + } + test("simple aggregation, state ver 1") { testStreamingAggregation(1) } @@ -796,4 +973,228 @@ abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Ass testForSide("right") } } + + protected def testSnapshotNotFound(): Unit = { + withTempDir { tempDir => + val provider = getNewStateStoreProvider(tempDir.getAbsolutePath) + for (i <- 1 to 4) { + val store = provider.getStore(i - 1) + put(store, "a", i, i) + store.commit() + provider.doMaintenance() // create a snapshot every other delta file + } + + val exc = intercept[SparkException] { + provider.asInstanceOf[SupportsFineGrainedReplay] + .replayReadStateFromSnapshot(1, 2) + } + checkError(exc, "CANNOT_LOAD_STATE_STORE.UNCATEGORIZED") + } + } + + protected def testGetReadStoreWithStartVersion(): Unit = { + withTempDir { tempDir => + val provider = getNewStateStoreProvider(tempDir.getAbsolutePath) + for (i <- 1 to 4) { + val store = provider.getStore(i - 1) + put(store, "a", i, i) + store.commit() + provider.doMaintenance() + } + + val result = + provider.asInstanceOf[SupportsFineGrainedReplay] + .replayReadStateFromSnapshot(2, 3) + + assert(get(result, "a", 1).get == 1) + assert(get(result, "a", 2).get == 2) + assert(get(result, "a", 3).get == 3) + assert(get(result, "a", 4).isEmpty) + + provider.close() + } + } + + protected def testSnapshotPartitionId(): Unit = { + withTempDir { tempDir => + val inputData = MemoryStream[Int] + val df = inputData.toDF().limit(10) + + testStream(df)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddData(inputData, 1, 2, 3, 4), + CheckLastBatch(1, 2, 3, 4) + ) + + val stateDf = spark.read.format("statestore") + .option(StateSourceOptions.SNAPSHOT_START_BATCH_ID, 0) + .option(StateSourceOptions.SNAPSHOT_PARTITION_ID, 0) + .option(StateSourceOptions.BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + + // should result in only one partition && should not throw error in planning stage + assert(stateDf.rdd.getNumPartitions == 1) + + // should throw error when partition id is out of range + val stateDfError = spark.read.format("statestore") + .option(StateSourceOptions.SNAPSHOT_START_BATCH_ID, 0) + .option( + StateSourceOptions.SNAPSHOT_PARTITION_ID, 1) + .option(StateSourceOptions.BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + + val exc = intercept[StateStoreSnapshotPartitionNotFound] { + stateDfError.show() + } + assert(exc.getErrorClass === "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND") + } + } + + private def testSnapshotStateDfAgainstStateDf(resourceDir: File): Unit = { + val stateSnapshotDf = spark.read.format("statestore") + .option("snapshotPartitionId", 0) + .option("snapshotStartBatchId", 1) + .load(resourceDir.getAbsolutePath) + + val stateDf = spark.read.format("statestore") + .load(resourceDir.getAbsolutePath) + .filter(col("partition_id") === 0) + + checkAnswer(stateSnapshotDf, stateDf) + } + + protected def testSnapshotOnLimitState(providerName: String): Unit = { + /** The golden files are generated by: + withSQLConf({ + SQLConf.STREAMING_MAINTENANCE_INTERVAL.key -> "100" + }) { + val inputData = MemoryStream[(Int, Long)] + val query = inputData.toDF().limit(10) + testStream(query)( + StartStream(checkpointLocation = <...>), + AddData(inputData, (1, 1L), (2, 2L), (3, 3L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (4, 4L), (5, 5L), (6, 6L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (7, 7L), (8, 8L), (9, 9L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (10, 10L), (11, 11L), (12, 12L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) } + ) + } + */ + val resourceUri = this.getClass.getResource( + s"/structured-streaming/checkpoint-version-4.0.0/$providerName/limit/" + ).toURI + + testSnapshotStateDfAgainstStateDf(new File(resourceUri)) + } + + protected def testSnapshotOnAggregateState(providerName: String): Unit = { + /** The golden files are generated by: + withSQLConf({ + SQLConf.STREAMING_MAINTENANCE_INTERVAL.key -> "100" + }) { + val inputData = MemoryStream[(Int, Long)] + val query = inputData.toDF().groupBy("_1").count() + testStream(query, OutputMode.Update)( + StartStream(checkpointLocation = <...>), + AddData(inputData, (1, 1L), (2, 2L), (3, 3L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (2, 2L), (3, 3L), (4, 4L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (3, 3L), (4, 4L), (5, 5L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (4, 4L), (5, 5L), (6, 6L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) } + ) + } + */ + val resourceUri = this.getClass.getResource( + s"/structured-streaming/checkpoint-version-4.0.0/$providerName/dedup/" + ).toURI + + testSnapshotStateDfAgainstStateDf(new File(resourceUri)) + } + + protected def testSnapshotOnDeduplicateState(providerName: String): Unit = { + /** The golden files are generated by: + withSQLConf({ + SQLConf.STREAMING_MAINTENANCE_INTERVAL.key -> "100" + }) { + val inputData = MemoryStream[(Int, Long)] + val query = inputData.toDF().dropDuplicates("_1") + testStream(query)( + StartStream(checkpointLocation = <...>), + AddData(inputData, (1, 1L), (2, 2L), (3, 3L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (2, 2L), (3, 3L), (4, 4L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (3, 3L), (4, 4L), (5, 5L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (4, 4L), (5, 5L), (6, 6L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) } + ) + } + */ + val resourceUri = this.getClass.getResource( + s"/structured-streaming/checkpoint-version-4.0.0/$providerName/dedup/" + ).toURI + + testSnapshotStateDfAgainstStateDf(new File(resourceUri)) + } + + protected def testSnapshotOnJoinState(providerName: String, stateVersion: Int): Unit = { + /** The golden files are generated by: + withSQLConf({ + SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString + SQLConf.STREAMING_MAINTENANCE_INTERVAL.key -> "100" + }) { + val inputData = MemoryStream[(Int, Long)] + val query = getStreamStreamJoinQuery(inputData) + testStream(query)( + StartStream(checkpointLocation = <...>), + AddData(inputData, (1, 1L), (2, 2L), (3, 3L), (4, 4L), (5, 5L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (6, 6L), (7, 7L), (8, 8L), (9, 9L), (10, 10L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) }, + AddData(inputData, (11, 11L), (12, 12L), (13, 13L), (14, 14L), (15, 15L)), + ProcessAllAvailable(), + Execute { _ => Thread.sleep(2000) } + ) + } + */ + val resourceUri = this.getClass.getResource( + s"/structured-streaming/checkpoint-version-4.0.0/$providerName/join$stateVersion/" + ).toURI + + val resourceDir = new File(resourceUri) + + val stateSnapshotDf = spark.read.format("statestore") + .option("snapshotPartitionId", 2) + .option("snapshotStartBatchId", 1) + .option("joinSide", "left") + .load(resourceDir.getAbsolutePath) + + val stateDf = spark.read.format("statestore") + .option("joinSide", "left") + .load(resourceDir.getAbsolutePath) + .filter(col("partition_id") === 2) + + checkAnswer(stateSnapshotDf, stateDf) + } }