ulysses-you commented on a change in pull request #767: URL: https://github.com/apache/incubator-kyuubi/pull/767#discussion_r669438033
########## File path: externals/kyuubi-spark-monitor/src/test/resources/test-scheduler-pool.xml ########## @@ -0,0 +1,30 @@ +<?xml version="1.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. + --> + +<allocations> + <pool name="p0"> + <minShare>2</minShare> + <weight>1</weight> + <schedulingMode>FAIR</schedulingMode> + </pool> + <pool name="p1"> + <minShare>0</minShare> + <weight>1</weight> + <schedulingMode>FAIR</schedulingMode> + </pool> +</allocations> Review comment: why we need this file ? ########## File path: externals/kyuubi-spark-monitor/src/main/scala/org/apache/kyuubi/engine/spark/monitor/KyuubiStatementMonitor.scala ########## @@ -0,0 +1,66 @@ +/* + * 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.kyuubi.engine.spark.monitor + +import java.util.concurrent.ArrayBlockingQueue + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.engine.spark.monitor.entity.KyuubiStatementInfo + +// TODO: Thread Safe need to consider +object KyuubiStatementMonitor extends Logging{ + + /** + * This blockingQueue store kyuubiStatementInfo. + * + * Notice: + * 1. When we remove items from this queue, we should ensure those statements have finished. + * If not, we should put them into this queue again. + * 2. There have two kinds of threshold to trigger when to remove items from this queue: + * a. time + * b. this queue's current size + */ + // TODO: Capacity should make configurable + private val kyuubiStatementQueue = new ArrayBlockingQueue[KyuubiStatementInfo](10) + + /** + * This function is used for putting kyuubiStatementInfo into blockingQueue(statementQueue). + * Every time we put an item into this queue, we should judge this queue's current size at first. + * If the size is less than threshold, we need to remove items from this queue. + * @param kyuubiStatementInfo + */ + // TODO: Lack size type threshold and time type threshold + def putStatementInfoIntoQueue(kyuubiStatementInfo: KyuubiStatementInfo): Unit = { + if (kyuubiStatementQueue.size() >= 7) { + removeAndDumpStatementInfoFromQueue() + } + val isSuccess = kyuubiStatementQueue.add(kyuubiStatementInfo) + info(s"Add kyuubiStatementInfo into queue is [$isSuccess], " + + s"statementId is [${kyuubiStatementInfo.statementId}]") + } + + /** + * This function is used for removing kyuubiStatementInfo from blockingQueue(statementQueue) + * and dumpping them to a file by threshold. + */ + // TODO: Need ensure those items have finished. If not, we should put them into this queue again. + def removeAndDumpStatementInfoFromQueue(): Unit = { Review comment: `private[kyuubi]` ########## File path: externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/KyuubiStatementMonitorSuite.scala ########## @@ -0,0 +1,86 @@ +/* + * 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.kyuubi.engine.spark + +import java.util.concurrent.ArrayBlockingQueue + +import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetOperationStatusReq, TOperationHandle} +import org.apache.hive.service.rpc.thrift.TCLIService.Iface +import org.apache.hive.service.rpc.thrift.TOperationState._ +import org.scalatest.PrivateMethodTester + +import org.apache.kyuubi.engine.spark.monitor.KyuubiStatementMonitor +import org.apache.kyuubi.engine.spark.monitor.entity.KyuubiStatementInfo +import org.apache.kyuubi.operation.HiveJDBCTests + +class KyuubiStatementMonitorSuite extends WithSparkSQLEngine with HiveJDBCTests + with PrivateMethodTester { + + override protected def jdbcUrl: String = getJdbcUrl + override def withKyuubiConf: Map[String, String] = Map.empty + + test("add kyuubiStatementInfo into queue and remove them by size type threshold") { + val sql = "select timestamp'2021-06-01'" + val total: Int = 7 + // Clear kyuubiStatementQueue first + val getQueue = PrivateMethod[ + ArrayBlockingQueue[KyuubiStatementInfo]](Symbol("kyuubiStatementQueue"))() + val kyuubiStatementQueue = KyuubiStatementMonitor.invokePrivate(getQueue) + kyuubiStatementQueue.clear() Review comment: is it better to add a new method `reset` in `KyuubiStatementMonitor` ? ########## File path: externals/kyuubi-spark-monitor/src/main/scala/org/apache/kyuubi/engine/spark/monitor/entity/KyuubiStatementInfo.scala ########## @@ -0,0 +1,46 @@ +/* + * 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.kyuubi.engine.spark.monitor.entity + +import scala.collection.mutable.Map + +import org.apache.spark.sql.execution.QueryExecution + +import org.apache.kyuubi.KyuubiSQLException +import org.apache.kyuubi.cli.HandleIdentifier +import org.apache.kyuubi.operation.OperationState.OperationState + +/** + * This object store the summary infomation about statement. + * You can use statementId to get all jobs' or stages' metric that this statement has. + * @param statementId + * @param statement + * @param appId + * @param sessionId + * @param queryExecution: contains physicalPlan, logicPlan and so on + * @param exception + * @param stateToTime: store this statement's every state and the time of occurrence + */ +case class KyuubiStatementInfo( + statementId: String, + statement: String, Review comment: let's say: `query: String` ########## File path: externals/kyuubi-spark-monitor/src/main/scala/org/apache/kyuubi/engine/spark/monitor/KyuubiStatementMonitor.scala ########## @@ -0,0 +1,66 @@ +/* + * 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.kyuubi.engine.spark.monitor + +import java.util.concurrent.ArrayBlockingQueue + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.engine.spark.monitor.entity.KyuubiStatementInfo + +// TODO: Thread Safe need to consider +object KyuubiStatementMonitor extends Logging{ Review comment: nit: `Logging{` -> `Logging {` ########## File path: externals/kyuubi-spark-monitor/src/main/scala/org/apache/kyuubi/engine/spark/monitor/entity/KyuubiStatementInfo.scala ########## @@ -0,0 +1,46 @@ +/* + * 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.kyuubi.engine.spark.monitor.entity + +import scala.collection.mutable.Map + +import org.apache.spark.sql.execution.QueryExecution + +import org.apache.kyuubi.KyuubiSQLException +import org.apache.kyuubi.cli.HandleIdentifier +import org.apache.kyuubi.operation.OperationState.OperationState + +/** + * This object store the summary infomation about statement. + * You can use statementId to get all jobs' or stages' metric that this statement has. + * @param statementId + * @param statement + * @param appId + * @param sessionId + * @param queryExecution: contains physicalPlan, logicPlan and so on + * @param exception + * @param stateToTime: store this statement's every state and the time of occurrence + */ +case class KyuubiStatementInfo( + statementId: String, + statement: String, + appId: String, + sessionId: HandleIdentifier, + var queryExecution: QueryExecution, + var exception: KyuubiSQLException, Review comment: can we move these `var` inside class ? ########## File path: externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/KyuubiStatementMonitorSuite.scala ########## @@ -0,0 +1,86 @@ +/* + * 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.kyuubi.engine.spark + +import java.util.concurrent.ArrayBlockingQueue + +import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetOperationStatusReq, TOperationHandle} +import org.apache.hive.service.rpc.thrift.TCLIService.Iface +import org.apache.hive.service.rpc.thrift.TOperationState._ +import org.scalatest.PrivateMethodTester + +import org.apache.kyuubi.engine.spark.monitor.KyuubiStatementMonitor +import org.apache.kyuubi.engine.spark.monitor.entity.KyuubiStatementInfo +import org.apache.kyuubi.operation.HiveJDBCTests + +class KyuubiStatementMonitorSuite extends WithSparkSQLEngine with HiveJDBCTests + with PrivateMethodTester { + + override protected def jdbcUrl: String = getJdbcUrl + override def withKyuubiConf: Map[String, String] = Map.empty + + test("add kyuubiStatementInfo into queue and remove them by size type threshold") { + val sql = "select timestamp'2021-06-01'" + val total: Int = 7 + // Clear kyuubiStatementQueue first + val getQueue = PrivateMethod[ + ArrayBlockingQueue[KyuubiStatementInfo]](Symbol("kyuubiStatementQueue"))() + val kyuubiStatementQueue = KyuubiStatementMonitor.invokePrivate(getQueue) + kyuubiStatementQueue.clear() + withSessionHandle { (client, handle) => + for ( a <- 1 to total ) { + val req = new TExecuteStatementReq() + req.setSessionHandle(handle) + req.setStatement(sql) + val tExecuteStatementResp = client.ExecuteStatement(req) + val operationHandle = tExecuteStatementResp.getOperationHandle + waitForOperationToComplete(client, operationHandle) + } + + var iterator = kyuubiStatementQueue.iterator() + while (iterator.hasNext) { + val kyuubiStatementInfo = iterator.next() + assert(kyuubiStatementInfo.statement !== null) Review comment: can we check the real value directly ? `!= null` seems unreliable ########## File path: externals/kyuubi-spark-monitor/src/main/scala/org/apache/kyuubi/engine/spark/monitor/KyuubiStatementMonitor.scala ########## @@ -0,0 +1,66 @@ +/* + * 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.kyuubi.engine.spark.monitor + +import java.util.concurrent.ArrayBlockingQueue + +import org.apache.kyuubi.Logging +import org.apache.kyuubi.engine.spark.monitor.entity.KyuubiStatementInfo + +// TODO: Thread Safe need to consider +object KyuubiStatementMonitor extends Logging{ + + /** + * This blockingQueue store kyuubiStatementInfo. + * + * Notice: + * 1. When we remove items from this queue, we should ensure those statements have finished. + * If not, we should put them into this queue again. + * 2. There have two kinds of threshold to trigger when to remove items from this queue: + * a. time + * b. this queue's current size + */ + // TODO: Capacity should make configurable + private val kyuubiStatementQueue = new ArrayBlockingQueue[KyuubiStatementInfo](10) + + /** + * This function is used for putting kyuubiStatementInfo into blockingQueue(statementQueue). + * Every time we put an item into this queue, we should judge this queue's current size at first. + * If the size is less than threshold, we need to remove items from this queue. + * @param kyuubiStatementInfo + */ + // TODO: Lack size type threshold and time type threshold + def putStatementInfoIntoQueue(kyuubiStatementInfo: KyuubiStatementInfo): Unit = { + if (kyuubiStatementQueue.size() >= 7) { Review comment: I'd like to define a constant: ``` val STATEMEMT_CAPACITY = 10 val FORCE_DUMP_STATEMENT_THREDHOLD = 7 ... ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
