luoyuxia commented on code in PR #19556: URL: https://github.com/apache/flink/pull/19556#discussion_r940469589
########## flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java: ########## @@ -500,19 +503,14 @@ public void testScriptTransform() throws Exception { try { // test explain transform String actualPlan = - (String) - CollectionUtil.iteratorToList( - tableEnv.executeSql( - "explain select transform(key, value)" - + " ROW FORMAT SERDE 'MySerDe'" - + " WITH SERDEPROPERTIES ('p1'='v1','p2'='v2')" - + " RECORDWRITER 'MyRecordWriter' " - + " using 'cat' as (cola int, value string)" - + " ROW FORMAT DELIMITED FIELDS TERMINATED BY ','" - + " RECORDREADER 'MyRecordReader' from src") - .collect()) - .get(0) - .getField(0); + explainSql( Review Comment: No any special reason. When working in this pr, I found there're some tests have the explain sql logic, so I extact the logic to a explain method, and then make other tests to call this method. ########## flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveSessionState.java: ########## @@ -0,0 +1,171 @@ +/* + * 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.planner.delegation.hive; + +import org.apache.flink.connectors.hive.FlinkHiveException; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; +import org.apache.flink.util.FileUtils; + +import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.lockmgr.LockException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.sql.Timestamp; +import java.time.Instant; + +/** Sub-class of SessionState to meet our needs. */ +public class HiveSessionState extends SessionState { + + private static final Logger LOG = LoggerFactory.getLogger(HiveSessionState.class); + + private static final Method setCurrentTSMethod = + HiveReflectionUtils.tryGetMethod( + SessionState.class, "setupQueryCurrentTimestamp", new Class[0]); + private static final Method getCurrentTSMethod = + HiveReflectionUtils.tryGetMethod( + SessionState.class, "getQueryCurrentTimestamp", new Class[0]); + + private static final Class registryClz; + private static final Method getRegistry; + private static final Method clearRegistry; + private static final Method closeRegistryLoaders; + + private Timestamp hiveParserCurrentTS; + + static { + registryClz = HiveReflectionUtils.tryGetClass("org.apache.hadoop.hive.ql.exec.Registry"); Review Comment: Actually, I'm not sure why use reflection here. I guess it's for compatibility for lower version hive like 1.1.0 since the class `Registry ` is introduced in [Hive 1.2.0](https://issues.apache.org/jira/browse/HIVE-2573). Although we only provide flink-sql-hive-connector 2.3.x and 3.1.x currenly, user may pack with other lower version hive-exec, so I think it'll be better not to change it. I think it's not necessary to put it in HiveShim. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org