Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali
// --------------------------------------------------------------------------------------------

override def snapshotConfiguration(): TypeSerializerConfigSnapshot[CRow] = {
new CRowSerializer.CRowSerializerConfigSnapshot(rowSerializer)
new CRowSerializer.CRowSerializerConfigSnapshot(Array(rowSerializer))
}

override def ensureCompatibility(
Expand Down Expand Up @@ -115,9 +115,13 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali

object CRowSerializer {

class CRowSerializerConfigSnapshot(rowSerializers: TypeSerializer[Row]*)
class CRowSerializerConfigSnapshot(rowSerializers: Array[TypeSerializer[Row]])
extends CompositeTypeSerializerConfigSnapshot[CRow](rowSerializers: _*) {

def this() {
this(Array.empty)
}

override def getVersion: Int = CRowSerializerConfigSnapshot.VERSION
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,20 @@

package org.apache.flink.table.runtime.types

import org.apache.flink.util.TestLogger
import org.junit.Test
import org.apache.flink.api.common.state.{ListState, ListStateDescriptor}
import org.apache.flink.api.common.typeinfo.Types
import org.apache.flink.api.java.functions.KeySelector
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.configuration.Configuration
import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend
import org.apache.flink.streaming.api.functions.KeyedProcessFunction
import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, KeyedProcessOperator}
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness
import org.apache.flink.types.Row
import org.apache.flink.util.{Collector, InstantiationUtil, TestLogger}

import org.junit.{Assert, Test}

class CRowSerializerTest extends TestLogger {

Expand All @@ -29,6 +41,70 @@ class CRowSerializerTest extends TestLogger {
@Test
def testDefaultConstructor(): Unit = {
new CRowSerializer.CRowSerializerConfigSnapshot()

InstantiationUtil.instantiate(classOf[CRowSerializer.CRowSerializerConfigSnapshot])
}

@Test
def testStateRestore(): Unit = {

class IKeyedProcessFunction extends KeyedProcessFunction[Integer, Integer, Integer] {
var state: ListState[CRow] = _
override def open(parameters: Configuration): Unit = {
val stateDesc = new ListStateDescriptor[CRow]("CRow",
new CRowTypeInfo(new RowTypeInfo(Types.INT)))
state = getRuntimeContext.getListState(stateDesc)
}
override def processElement(value: Integer,
ctx: KeyedProcessFunction[Integer, Integer, Integer]#Context,
out: Collector[Integer]): Unit = {
state.add(new CRow(Row.of(value), true))
}
}

val operator = new KeyedProcessOperator[Integer, Integer, Integer](new IKeyedProcessFunction)

var testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, Integer](
operator,
new KeySelector[Integer, Integer] {
override def getKey(value: Integer): Integer= -1
},
Types.INT, 1, 1, 0)
testHarness.setup()
testHarness.open()

testHarness.processElement(new StreamRecord[Integer](1, 1L))
testHarness.processElement(new StreamRecord[Integer](2, 1L))
testHarness.processElement(new StreamRecord[Integer](3, 1L))

Assert.assertEquals(1, numKeyedStateEntries(operator))

val snapshot = testHarness.snapshot(0L, 0L)
testHarness.close()

testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, Integer](
operator,
new KeySelector[Integer, Integer] {
override def getKey(value: Integer): Integer= -1
},
Types.INT, 1, 1, 0)
testHarness.setup()

testHarness.initializeState(snapshot)

testHarness.open()

Assert.assertEquals(1, numKeyedStateEntries(operator))

testHarness.close()
}

def numKeyedStateEntries(operator: AbstractStreamOperator[_]): Int = {
val keyedStateBackend = operator.getKeyedStateBackend
keyedStateBackend match {
case hksb: HeapKeyedStateBackend[_] => hksb.numKeyValueStateEntries
case _ => throw new UnsupportedOperationException
}
}

}