This is an automated email from the ASF dual-hosted git repository.
jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new a70bd35 [FLINK-12911][table-api] Port AppendStreamTableSink,
UpsertStreamTableSink, RetractStreamTableSink to flink-api-java-bridge
a70bd35 is described below
commit a70bd35e306640ae813813294cc2516b96647e73
Author: Jark Wu <[email protected]>
AuthorDate: Fri Jun 21 11:12:00 2019 +0800
[FLINK-12911][table-api] Port AppendStreamTableSink, UpsertStreamTableSink,
RetractStreamTableSink to flink-api-java-bridge
This closes #8819
---
.../flink/table/sinks/AppendStreamTableSink.java} | 27 +++----
.../flink/table/sinks/RetractStreamTableSink.java | 54 ++++++++++++++
.../flink/table/sinks/UpsertStreamTableSink.java | 84 ++++++++++++++++++++++
.../flink/table/sinks/AppendStreamTableSink.scala | 32 ---------
.../flink/table/sinks/RetractStreamTableSink.scala | 55 --------------
.../flink/table/sinks/UpsertStreamTableSink.scala | 80 ---------------------
.../flink/table/sinks/RetractStreamTableSink.scala | 55 --------------
.../flink/table/sinks/UpsertStreamTableSink.scala | 80 ---------------------
8 files changed, 152 insertions(+), 315 deletions(-)
diff --git
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/AppendStreamTableSink.java
similarity index 55%
rename from
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
rename to
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/AppendStreamTableSink.java
index d73380e..52059c2 100644
---
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
+++
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/AppendStreamTableSink.java
@@ -16,21 +16,22 @@
* limitations under the License.
*/
-package org.apache.flink.table.sinks
+package org.apache.flink.table.sinks;
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.Table
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
/**
- * Defines an external [[TableSink]] to emit streaming [[Table]] with only
insert changes.
- *
- * If the [[Table]] is also modified by update or delete changes, a
- * [[org.apache.flink.table.api.TableException]] will be thrown.
- *
- * @tparam T Type of [[DataStream]] that this [[TableSink]] expects and
supports.
- */
-trait AppendStreamTableSink[T] extends StreamTableSink[T] {
+ * Defines an external {@link TableSink} to emit streaming {@link Table} with
only insert changes.
+ *
+ * <p>If the {@link Table} is also modified by update or delete changes, a
{@link TableException}
+ * will be thrown.
+ *
+ * @param <T> Type of {@link DataStream} that this {@link TableSink} expects
and supports.
+ */
+@PublicEvolving
+public interface AppendStreamTableSink<T> extends StreamTableSink<T> {
- /** Emits the DataStream. */
- def emitDataStream(dataStream: DataStream[T]): Unit
}
diff --git
a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/RetractStreamTableSink.java
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/RetractStreamTableSink.java
new file mode 100644
index 0000000..6660e7c
--- /dev/null
+++
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/RetractStreamTableSink.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.table.api.Table;
+
+/**
+ * Defines an external {@link TableSink} to emit a streaming {@link Table}
with insert, update,
+ * and delete changes.
+ *
+ * <p>The table will be converted into a stream of accumulate and retraction
messages which are
+ * encoded as {@link Tuple2}. The first field is a {@link Boolean} flag to
indicate the message type.
+ * The second field holds the record of the requested type {@link T}.
+ *
+ * <p>A message with true {@link Boolean} flag is an accumulate (or add)
message.
+ *
+ * <p>A message with false flag is a retract message.
+ *
+ * @param <T> Type of records that this {@link TableSink} expects and supports.
+ */
+@PublicEvolving
+public interface RetractStreamTableSink<T> extends
StreamTableSink<Tuple2<Boolean, T>> {
+
+ /**
+ * Returns the requested record type.
+ */
+ TypeInformation<T> getRecordType();
+
+ @Override
+ default TypeInformation<Tuple2<Boolean, T>> getOutputType() {
+ return new TupleTypeInfo<>(Types.BOOLEAN, getRecordType());
+ }
+}
diff --git
a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/UpsertStreamTableSink.java
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/UpsertStreamTableSink.java
new file mode 100644
index 0000000..3da5ee1
--- /dev/null
+++
b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/UpsertStreamTableSink.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
+
+/**
+ * Defines an external {@link TableSink} to emit a streaming {@link Table}
with insert, update,
+ * and delete changes. The {@link Table} must be have unique key fields
(atomic or composite)
+ * or be append-only.
+ *
+ * <p>If the {@link Table} does not have a unique key and is not append-only,
a {@link TableException}
+ * will be thrown.
+ *
+ * <p>The unique key of the table is configured by the {@link
UpsertStreamTableSink#setKeyFields(String[])}
+ * method.
+ *
+ * <p>The {@link Table} will be converted into a stream of upsert and delete
messages which are
+ * encoded as {@link Tuple2}. The first field is a {@link Boolean} flag to
indicate the message
+ * type. The second field holds the record of the requested type {@link T}.
+ *
+ * <p>A message with true {@link Boolean} field is an upsert message for the
configured key.
+ *
+ * <p>A message with false flag is a delete message for the configured key.
+ *
+ * <p>If the table is append-only, all messages will have a true flag and must
be interpreted
+ * as insertions.
+ *
+ * @param <T> Type of records that this {@link TableSink} expects and supports.
+ */
+@PublicEvolving
+public interface UpsertStreamTableSink<T> extends
StreamTableSink<Tuple2<Boolean, T>> {
+
+ /**
+ * Configures the unique key fields of the {@link Table} to write.
+ * The method is called after {@link TableSink#configure(String[],
TypeInformation[])}.
+ *
+ * <p>The keys array might be empty, if the table consists of a single
(updated) record.
+ * If the table does not have a key and is append-only, the keys
attribute is null.
+ *
+ * @param keys the field names of the table's keys, an empty array if
the table has a single
+ * row, and null if the table is append-only and has no key.
+ */
+ void setKeyFields(String[] keys);
+
+ /**
+ * Specifies whether the {@link Table} to write is append-only or not.
+ *
+ * @param isAppendOnly true if the table is append-only, false
otherwise.
+ */
+ void setIsAppendOnly(Boolean isAppendOnly);
+
+ /**
+ * Returns the requested record type.
+ */
+ TypeInformation<T> getRecordType();
+
+ @Override
+ default TypeInformation<Tuple2<Boolean, T>> getOutputType() {
+ return new TupleTypeInfo<>(Types.BOOLEAN, getRecordType());
+ }
+}
diff --git
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
deleted file mode 100644
index d0c8807..0000000
---
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.sinks
-
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.Table
-
-/**
- * Defines an external [[TableSink]] to emit streaming [[Table]] with only
insert changes.
- *
- * If the [[Table]] is also modified by update or delete changes, a
- * [[org.apache.flink.table.api.TableException]] will be thrown.
- *
- * @tparam T Type of [[DataStream]] that this [[TableSink]] expects and
supports.
- */
-trait AppendStreamTableSink[T] extends StreamTableSink[T]
diff --git
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
deleted file mode 100644
index 2d7c46f..0000000
---
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.sinks
-
-import java.lang.{Boolean => JBool}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TupleTypeInfo
-import org.apache.flink.table.api.Types
-
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.Table
-
-/**
- * Defines an external [[TableSink]] to emit a streaming [[Table]] with
insert, update, and delete
- * changes.
- *
- * The table will be converted into a stream of accumulate and retraction
messages which are
- * encoded as [[JTuple2]].
- * The first field is a [[JBool]] flag to indicate the message type.
- * The second field holds the record of the requested type [[T]].
- *
- * A message with true [[JBool]] flag is an accumulate (or add) message.
- * A message with false flag is a retract message.
- *
- * @tparam T Type of records that this [[TableSink]] expects and supports.
- */
-trait RetractStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
-
- /** Returns the requested record type */
- def getRecordType: TypeInformation[T]
-
- /** Emits the DataStream. */
- def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
-
- override def getOutputType = new TupleTypeInfo(Types.BOOLEAN, getRecordType)
-
-}
diff --git
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
deleted file mode 100644
index d3be507..0000000
---
a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.sinks
-
-import java.lang.{Boolean => JBool}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.api.java.typeutils.TupleTypeInfo
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.{Table, Types}
-
-/**
- * Defines an external [[TableSink]] to emit a streaming [[Table]] with
insert, update, and delete
- * changes. The [[Table]] must be have unique key fields (atomic or
composite) or be append-only.
- *
- * If the [[Table]] does not have a unique key and is not append-only, a
- * [[org.apache.flink.table.api.TableException]] will be thrown.
- *
- * The unique key of the table is configured by the
[[UpsertStreamTableSink#setKeyFields()]]
- * method.
- *
- * The [[Table]] will be converted into a stream of upsert and delete
messages which are encoded as
- * [[JTuple2]]. The first field is a [[JBool]] flag to indicate the message
type. The second field
- * holds the record of the requested type [[T]].
- *
- * A message with true [[JBool]] field is an upsert message for the
configured key.
- * A message with false flag is a delete message for the configured key.
- *
- * If the table is append-only, all messages will have a true flag and must
be interpreted
- * as insertions.
- *
- * @tparam T Type of records that this [[TableSink]] expects and supports.
- */
-trait UpsertStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
-
- /**
- * Configures the unique key fields of the [[Table]] to write.
- * The method is called after [[TableSink.configure()]].
- *
- * The keys array might be empty, if the table consists of a single
(updated) record.
- * If the table does not have a key and is append-only, the keys attribute
is null.
- *
- * @param keys the field names of the table's keys, an empty array if the
table has a single
- * row, and null if the table is append-only and has no key.
- */
- def setKeyFields(keys: Array[String]): Unit
-
- /**
- * Specifies whether the [[Table]] to write is append-only or not.
- *
- * @param isAppendOnly true if the table is append-only, false otherwise.
- */
- def setIsAppendOnly(isAppendOnly: JBool): Unit
-
- /** Returns the requested record type */
- def getRecordType: TypeInformation[T]
-
- /** Emits the DataStream. */
- def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
-
- override def getOutputType: TypeInformation[JTuple2[JBool, T]] =
- new TupleTypeInfo(Types.BOOLEAN, getRecordType)
-}
diff --git
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
deleted file mode 100644
index 2d7c46f..0000000
---
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.sinks
-
-import java.lang.{Boolean => JBool}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TupleTypeInfo
-import org.apache.flink.table.api.Types
-
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.Table
-
-/**
- * Defines an external [[TableSink]] to emit a streaming [[Table]] with
insert, update, and delete
- * changes.
- *
- * The table will be converted into a stream of accumulate and retraction
messages which are
- * encoded as [[JTuple2]].
- * The first field is a [[JBool]] flag to indicate the message type.
- * The second field holds the record of the requested type [[T]].
- *
- * A message with true [[JBool]] flag is an accumulate (or add) message.
- * A message with false flag is a retract message.
- *
- * @tparam T Type of records that this [[TableSink]] expects and supports.
- */
-trait RetractStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
-
- /** Returns the requested record type */
- def getRecordType: TypeInformation[T]
-
- /** Emits the DataStream. */
- def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
-
- override def getOutputType = new TupleTypeInfo(Types.BOOLEAN, getRecordType)
-
-}
diff --git
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
deleted file mode 100644
index d3be507..0000000
---
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.sinks
-
-import java.lang.{Boolean => JBool}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
-import org.apache.flink.api.java.typeutils.TupleTypeInfo
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.{Table, Types}
-
-/**
- * Defines an external [[TableSink]] to emit a streaming [[Table]] with
insert, update, and delete
- * changes. The [[Table]] must be have unique key fields (atomic or
composite) or be append-only.
- *
- * If the [[Table]] does not have a unique key and is not append-only, a
- * [[org.apache.flink.table.api.TableException]] will be thrown.
- *
- * The unique key of the table is configured by the
[[UpsertStreamTableSink#setKeyFields()]]
- * method.
- *
- * The [[Table]] will be converted into a stream of upsert and delete
messages which are encoded as
- * [[JTuple2]]. The first field is a [[JBool]] flag to indicate the message
type. The second field
- * holds the record of the requested type [[T]].
- *
- * A message with true [[JBool]] field is an upsert message for the
configured key.
- * A message with false flag is a delete message for the configured key.
- *
- * If the table is append-only, all messages will have a true flag and must
be interpreted
- * as insertions.
- *
- * @tparam T Type of records that this [[TableSink]] expects and supports.
- */
-trait UpsertStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
-
- /**
- * Configures the unique key fields of the [[Table]] to write.
- * The method is called after [[TableSink.configure()]].
- *
- * The keys array might be empty, if the table consists of a single
(updated) record.
- * If the table does not have a key and is append-only, the keys attribute
is null.
- *
- * @param keys the field names of the table's keys, an empty array if the
table has a single
- * row, and null if the table is append-only and has no key.
- */
- def setKeyFields(keys: Array[String]): Unit
-
- /**
- * Specifies whether the [[Table]] to write is append-only or not.
- *
- * @param isAppendOnly true if the table is append-only, false otherwise.
- */
- def setIsAppendOnly(isAppendOnly: JBool): Unit
-
- /** Returns the requested record type */
- def getRecordType: TypeInformation[T]
-
- /** Emits the DataStream. */
- def emitDataStream(dataStream: DataStream[JTuple2[JBool, T]]): Unit
-
- override def getOutputType: TypeInformation[JTuple2[JBool, T]] =
- new TupleTypeInfo(Types.BOOLEAN, getRecordType)
-}