You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/28 09:49:08 UTC

[GitHub] [flink-table-store] JingsongLi opened a new pull request, #179: [FLINK-28282] Planner free in flink-table-store-codegen

JingsongLi opened a new pull request, #179:
URL: https://github.com/apache/flink-table-store/pull/179

   We currently have the table-planner bundled into flink-table-store-codegen, which causes:
   - bundle jar is too big, 20+MB
   - Dependence on planner code will make it difficult to be compatible with multiple versions of Flink


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi merged pull request #179: [FLINK-28282] Planner free in flink-table-store-codegen

Posted by GitBox <gi...@apache.org>.
JingsongLi merged PR #179:
URL: https://github.com/apache/flink-table-store/pull/179


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #179: [FLINK-28282] Planner free in flink-table-store-codegen

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on code in PR #179:
URL: https://github.com/apache/flink-table-store/pull/179#discussion_r909617541


##########
flink-table-store-codegen/src/main/scala/org.apache.flink.table.store.codegen/CodeGeneratorContext.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.store.codegen
+
+import org.apache.flink.table.runtime.typeutils.InternalSerializers
+import org.apache.flink.table.store.codegen.GenerateUtils.{newName, newNames}
+import org.apache.flink.table.types.logical.LogicalType
+import org.apache.flink.util.InstantiationUtil
+
+import scala.collection.mutable
+
+/**
+ * The context for code generator, maintaining various reusable statements that could be insert into
+ * different code sections in the final generated class.
+ */
+class CodeGeneratorContext {
+
+  val classLoader: ClassLoader = Thread.currentThread().getContextClassLoader
+
+  // holding a list of objects that could be used passed into generated class
+  val references: mutable.ArrayBuffer[AnyRef] = new mutable.ArrayBuffer[AnyRef]()
+
+  // set of member statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableMemberStatements: mutable.LinkedHashSet[String] =
+  mutable.LinkedHashSet[String]()
+
+  // set of constructor statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableInitStatements: mutable.LinkedHashSet[String] =
+  mutable.LinkedHashSet[String]()
+
+  // map of type serializer that will be added only once
+  // LogicalType -> reused_term
+  private val reusableTypeSerializers: mutable.Map[LogicalType, String] =
+  mutable.Map[LogicalType, String]()
+
+  // local variable statements.
+  private val reusableLocalVariableStatements = mutable.LinkedHashSet[String]()
+
+  /**
+   * Adds multiple pairs of local variables. The local variable statements will be placed in methods
+   * or class member area depends on whether the method length excess max code length.
+   *
+   * @param fieldTypeAndNames
+   *   pairs of local variables with left is field type term and right is field name
+   * @return
+   *   the new generated unique field names for each variable pairs
+   */
+  def addReusableLocalVariables(fieldTypeAndNames: (String, String)*): Seq[String] = {
+    val fieldTerms = newNames(fieldTypeAndNames.map(_._2): _*)
+    fieldTypeAndNames.map(_._1).zip(fieldTerms).foreach {
+      case (fieldTypeTerm, fieldTerm) =>
+        reusableLocalVariableStatements.add(s"$fieldTypeTerm $fieldTerm;")
+    }
+    fieldTerms
+  }
+
+  /**
+   * Adds a reusable member field statement to the member area.
+   *
+   * @param memberStatement
+   *   the member field declare statement
+   */
+  def addReusableMember(memberStatement: String): Unit = {
+    reusableMemberStatements.add(memberStatement)
+  }
+
+  /**
+   * Adds a reusable Object to the member area of the generated class
+   * @param obj
+   *   the object to be added to the generated class
+   * @param fieldNamePrefix
+   *   prefix field name of the generated member field term
+   * @param fieldTypeTerm
+   *   field type class name
+   * @return
+   *   the generated unique field term
+   */
+  def addReusableObject(
+                         obj: AnyRef,
+                         fieldNamePrefix: String,
+                         fieldTypeTerm: String = null): String = {

Review Comment:
   Let's do it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] tsreaper commented on a diff in pull request #179: [FLINK-28282] Planner free in flink-table-store-codegen

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #179:
URL: https://github.com/apache/flink-table-store/pull/179#discussion_r909404056


##########
flink-table-store-codegen/src/main/scala/org.apache.flink.table.store.codegen/CodeGeneratorContext.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.store.codegen
+
+import org.apache.flink.table.runtime.typeutils.InternalSerializers
+import org.apache.flink.table.store.codegen.GenerateUtils.{newName, newNames}
+import org.apache.flink.table.types.logical.LogicalType
+import org.apache.flink.util.InstantiationUtil
+
+import scala.collection.mutable
+
+/**
+ * The context for code generator, maintaining various reusable statements that could be insert into
+ * different code sections in the final generated class.
+ */
+class CodeGeneratorContext {
+
+  val classLoader: ClassLoader = Thread.currentThread().getContextClassLoader
+
+  // holding a list of objects that could be used passed into generated class
+  val references: mutable.ArrayBuffer[AnyRef] = new mutable.ArrayBuffer[AnyRef]()
+
+  // set of member statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableMemberStatements: mutable.LinkedHashSet[String] =
+  mutable.LinkedHashSet[String]()
+
+  // set of constructor statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableInitStatements: mutable.LinkedHashSet[String] =
+  mutable.LinkedHashSet[String]()
+
+  // map of type serializer that will be added only once
+  // LogicalType -> reused_term
+  private val reusableTypeSerializers: mutable.Map[LogicalType, String] =
+  mutable.Map[LogicalType, String]()
+
+  // local variable statements.
+  private val reusableLocalVariableStatements = mutable.LinkedHashSet[String]()
+
+  /**
+   * Adds multiple pairs of local variables. The local variable statements will be placed in methods
+   * or class member area depends on whether the method length excess max code length.
+   *
+   * @param fieldTypeAndNames
+   *   pairs of local variables with left is field type term and right is field name
+   * @return
+   *   the new generated unique field names for each variable pairs
+   */
+  def addReusableLocalVariables(fieldTypeAndNames: (String, String)*): Seq[String] = {
+    val fieldTerms = newNames(fieldTypeAndNames.map(_._2): _*)
+    fieldTypeAndNames.map(_._1).zip(fieldTerms).foreach {
+      case (fieldTypeTerm, fieldTerm) =>
+        reusableLocalVariableStatements.add(s"$fieldTypeTerm $fieldTerm;")
+    }
+    fieldTerms
+  }
+
+  /**
+   * Adds a reusable member field statement to the member area.
+   *
+   * @param memberStatement
+   *   the member field declare statement
+   */
+  def addReusableMember(memberStatement: String): Unit = {
+    reusableMemberStatements.add(memberStatement)
+  }
+
+  /**
+   * Adds a reusable Object to the member area of the generated class
+   * @param obj
+   *   the object to be added to the generated class
+   * @param fieldNamePrefix
+   *   prefix field name of the generated member field term
+   * @param fieldTypeTerm
+   *   field type class name
+   * @return
+   *   the generated unique field term
+   */
+  def addReusableObject(
+                         obj: AnyRef,
+                         fieldNamePrefix: String,
+                         fieldTypeTerm: String = null): String = {

Review Comment:
   Fix indent please. Shall we also introduce scala style plugin?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-table-store] tsreaper commented on a diff in pull request #179: [FLINK-28282] Planner free in flink-table-store-codegen

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #179:
URL: https://github.com/apache/flink-table-store/pull/179#discussion_r910558828


##########
flink-table-store-codegen/src/main/scala/org.apache.flink.table.store.codegen/ExprCodeGenerator.scala:
##########
@@ -0,0 +1,171 @@
+/*
+                                    * Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Fix indent please.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org