From 634991c1f454d8b99a55ac80fa60c1ce81419ec4 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Mon, 9 Dec 2024 10:57:54 +0800 Subject: [PATCH 01/38] init module kyuubi-spark-connector-yarn --- .../spark/kyuubi-spark-connector-yarn/pom.xml | 230 ++++++++++++++++++ .../spark/connector/yarn/YarnCatalog.scala | 69 ++++++ .../connector/yarn/YarnLogBatchScan.scala | 34 +++ .../spark/connector/yarn/YarnLogsTable.scala | 42 ++++ pom.xml | 1 + 5 files changed, 376 insertions(+) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/pom.xml create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml new file mode 100644 index 00000000000..83436204a27 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml @@ -0,0 +1,230 @@ + + + + 4.0.0 + + org.apache.kyuubi + kyuubi-parent + 1.11.0-SNAPSHOT + ../../../pom.xml + + + kyuubi-spark-connector-yarn_${scala.binary.version} + jar + Kyuubi Spark Yarn Logs Connector + https://kyuubi.apache.org/ + + + + org.apache.kyuubi + kyuubi-spark-connector-common_${scala.binary.version} + ${project.version} + + + + org.apache.kyuubi + kyuubi-spark-connector-common_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.scala-lang + scala-library + provided + + + + org.slf4j + slf4j-api + provided + + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + + com.google.guava + guava + + + + org.apache.spark + spark-core_${scala.binary.version} + test-jar + test + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + test + + + + org.scalatestplus + scalacheck-1-17_${scala.binary.version} + test + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + test-jar + test + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + test + + + + org.apache.kyuubi + kyuubi-common_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.hadoop + hadoop-client-api + test + + + + org.apache.hadoop + hadoop-client-runtime + test + + + + javax.servlet + javax.servlet-api + test + + + + jakarta.servlet + jakarta.servlet-api + test + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + true + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + com.google.guava:guava + org.apache.kyuubi:* + + + + + com.google.common + ${kyuubi.shade.packageName}.com.google.common + + com.google.common.** + + + + + + + + shade + + package + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + cross-version-test + + + + org.apache.maven.plugins + maven-clean-plugin + + true + + + target/scala-${scala.binary.version}/classes + **/*.* + + + + + + clean target/scala-${scala.binary.version}/classes + + clean + + process-test-classes + + + + + + + + diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala new file mode 100644 index 00000000000..dc2bdbb047e --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -0,0 +1,69 @@ +/* + * 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.spark.connector.yarn + +import java.util + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { + private var catalogName: String = _ + + override def initialize( + name: String, + caseInsensitiveStringMap: CaseInsensitiveStringMap): Unit = { + catalogName = name + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + Array(Identifier.of(namespace, "agg_logs")) + } + + override def loadTable(identifier: Identifier): Table = ??? + + override def createTable( + identifier: Identifier, + structType: StructType, + transforms: Array[Transform], + map: util.Map[String, String]): Table = ??? + + override def alterTable(identifier: Identifier, tableChanges: TableChange*): Table = ??? + + override def dropTable(identifier: Identifier): Boolean = ??? + + override def renameTable(identifier: Identifier, identifier1: Identifier): Unit = ??? + + override def listNamespaces(): Array[Array[String]] = ??? + + override def listNamespaces(strings: Array[String]): Array[Array[String]] = ??? + + override def loadNamespaceMetadata(strings: Array[String]): util.Map[String, String] = ??? + + override def createNamespace(strings: Array[String], map: util.Map[String, String]): Unit = ??? + + override def alterNamespace(strings: Array[String], namespaceChanges: NamespaceChange*): Unit = + ??? + + override def dropNamespace(strings: Array[String], b: Boolean): Boolean = ??? + + override def name(): String = this.catalogName +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala new file mode 100644 index 00000000000..bacb011b427 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala @@ -0,0 +1,34 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class YarnLogsScan(options: CaseInsensitiveStringMap, schema: StructType) extends Scan with Batch { + private val appId: String = options.getOrDefault("appId", "*") + override def readSchema(): StructType = schema + + override def planInputPartitions(): Array[InputPartition] = { + // Fetch logs for the given appId (filtering logic can be added) + Array(new YarnLogsPartition(appId)) + } + + override def createReaderFactory(): PartitionReaderFactory = ??? +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala new file mode 100644 index 00000000000..4ed4e7f5341 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala @@ -0,0 +1,42 @@ +/* + * 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.spark.connector.yarn + +import java.util + +import scala.jdk.CollectionConverters.setAsJavaSetConverter + +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class YarnLogsTable extends Table with SupportsRead { + override def name(): String = "agg_logs" + + override def schema(): StructType = + new StructType(Array( + StructField("appId", StringType, nullable = false), + StructField("logLevel", StringType, nullable = false), + StructField("message", StringType, nullable = true))) + + override def capabilities(): util.Set[TableCapability] = + Set(TableCapability.BATCH_READ).asJava + + override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = ??? +} diff --git a/pom.xml b/pom.xml index 73146363243..89376e4b4f3 100644 --- a/pom.xml +++ b/pom.xml @@ -61,6 +61,7 @@ extensions/spark/kyuubi-spark-connector-common extensions/spark/kyuubi-spark-connector-tpcds extensions/spark/kyuubi-spark-connector-tpch + extensions/spark/kyuubi-spark-connector-yarn extensions/spark/kyuubi-spark-lineage extensions/spark/kyuubi-spark-jvm-quake externals/kyuubi-chat-engine From 855c5e00fbced3172e5d99acaf85f5a7adad2eea Mon Sep 17 00:00:00 2001 From: native-zhang Date: Wed, 11 Dec 2024 00:51:28 +0800 Subject: [PATCH 02/38] introduce DFSMiniCluster and YarnCluster for test cases --- .../spark/kyuubi-spark-connector-yarn/pom.xml | 45 ++++ .../spark/connector/yarn/YarnCatalog.scala | 61 ++++-- .../connector/yarn/YarnLogsPartition.scala | 22 ++ .../yarn/YarnLogsPartitionReader.scala | 47 ++++ .../yarn/YarnLogsReaderFactory.scala | 29 +++ ...nLogBatchScan.scala => YarnLogsScan.scala} | 10 +- .../spark/connector/yarn/YarnLogsTable.scala | 15 +- .../yarn/SparkYarnConnectorOnYarn.scala | 63 ++++++ .../WithKyuubiServerAndYarnMiniCluster.scala | 205 ++++++++++++++++++ 9 files changed, 475 insertions(+), 22 deletions(-) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/{YarnLogBatchScan.scala => YarnLogsScan.scala} (84%) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml index 83436204a27..6ee869b5aee 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml +++ b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml @@ -96,6 +96,21 @@ test + + org.apache.kyuubi + kyuubi-server_${scala.binary.version} + ${project.version} + test + + + + org.apache.kyuubi + kyuubi-server_${scala.binary.version} + ${project.version} + test-jar + test + + org.apache.kyuubi kyuubi-common_${scala.binary.version} @@ -123,6 +138,12 @@ test + + org.apache.hadoop + hadoop-client-minicluster + test + + javax.servlet javax.servlet-api @@ -134,6 +155,30 @@ jakarta.servlet-api test + + + org.bouncycastle + bcprov-jdk18on + test + + + + org.bouncycastle + bcpkix-jdk18on + test + + + + jakarta.activation + jakarta.activation-api + test + + + + jakarta.xml.bind + jakarta.xml.bind-api + test + diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index dc2bdbb047e..29dd04db89f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -19,7 +19,10 @@ package org.apache.kyuubi.spark.connector.yarn import java.util +import scala.jdk.CollectionConverters.mapAsJavaMapConverter + import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -35,35 +38,67 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { } override def listTables(namespace: Array[String]): Array[Identifier] = { - Array(Identifier.of(namespace, "agg_logs")) + Array(Identifier.of(namespace, "app_logs"), Identifier.of(namespace, "apps")) } - override def loadTable(identifier: Identifier): Table = ??? + override def loadTable(identifier: Identifier): Table = identifier.name match { + case "app_logs" => new YarnLogsTable + // TODO impl YarnAppTable + // case "apps" => new YarnAppTable + case _ => throw new UnsupportedOperationException(s"Table ${identifier.name()} not found") + + } override def createTable( identifier: Identifier, structType: StructType, transforms: Array[Transform], - map: util.Map[String, String]): Table = ??? + map: util.Map[String, String]): Table = { + throw new UnsupportedOperationException("Create table is not supported") + } + + override def alterTable(identifier: Identifier, tableChanges: TableChange*): Table = { + throw new UnsupportedOperationException("Alter table is not supported") + } - override def alterTable(identifier: Identifier, tableChanges: TableChange*): Table = ??? + override def dropTable(identifier: Identifier): Boolean = { + throw new UnsupportedOperationException("Drop table is not supported") + } - override def dropTable(identifier: Identifier): Boolean = ??? + override def renameTable(identifier: Identifier, identifier1: Identifier): Unit = { + throw new UnsupportedOperationException("Rename table is not supported") + } - override def renameTable(identifier: Identifier, identifier1: Identifier): Unit = ??? + override def listNamespaces(): Array[Array[String]] = { + Array("default") + } - override def listNamespaces(): Array[Array[String]] = ??? + override def listNamespaces(namespace: Array[String]): Array[Array[String]] = namespace match { + case Array() => listNamespaces() + // TODO make it available + case Array(db) if db eq "default" => listNamespaces() + case _ => throw new NoSuchNamespaceException(namespace) + } - override def listNamespaces(strings: Array[String]): Array[Array[String]] = ??? + override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = + namespace match { + case Array(_) => Map.empty[String, String].asJava + case _ => throw new NoSuchNamespaceException(namespace) + } - override def loadNamespaceMetadata(strings: Array[String]): util.Map[String, String] = ??? + override def createNamespace(namespace: Array[String], metadata: util.Map[String, String]): Unit = + throw new UnsupportedOperationException - override def createNamespace(strings: Array[String], map: util.Map[String, String]): Unit = ??? + override def alterNamespace(namespace: Array[String], changes: NamespaceChange*): Unit = + throw new UnsupportedOperationException - override def alterNamespace(strings: Array[String], namespaceChanges: NamespaceChange*): Unit = - ??? + // Removed in SPARK-37929 + def dropNamespace(namespace: Array[String]): Boolean = + throw new UnsupportedOperationException - override def dropNamespace(strings: Array[String], b: Boolean): Boolean = ??? + // Introduced in SPARK-37929 + def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean = + throw new UnsupportedOperationException override def name(): String = this.catalogName } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala new file mode 100644 index 00000000000..5cc61b6cb4d --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read.InputPartition + +class YarnLogsPartition(val appId: String) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala new file mode 100644 index 00000000000..6e4b6899fd8 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala @@ -0,0 +1,47 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.connector.read.PartitionReader + +class YarnLogsPartitionReader(appId: String) extends PartitionReader[InternalRow] { + + private val logsIterator = fetchLogs(appId).iterator + + override def next(): Boolean = logsIterator.hasNext + + override def get(): InternalRow = { + val log = logsIterator.next() + new GenericInternalRow(Array[Any](log.appId, log.logLevel, log.message)) + } + + override def close(): Unit = {} + + private def fetchLogs(appId: String): Seq[LogEntry] = { + // Simulate fetching logs for the given appId (replace with Yarn API calls) + Seq( + LogEntry(appId, "INFO", "Application started"), + LogEntry(appId, "WARN", "Low memory"), + LogEntry(appId, "ERROR", "Application failed")) + } +} + +// Helper class to represent log entries +case class LogEntry(appId: String, logLevel: String, message: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala new file mode 100644 index 00000000000..1d7d97d0ed6 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala @@ -0,0 +1,29 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} + +class YarnLogsReaderFactory extends PartitionReaderFactory { + + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + val yarnPartition = partition.asInstanceOf[YarnLogsPartition] + new YarnLogsPartitionReader(yarnPartition.appId) + } +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala similarity index 84% rename from extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala index bacb011b427..fd680189d82 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogBatchScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala @@ -17,11 +17,12 @@ package org.apache.kyuubi.spark.connector.yarn -import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnLogsScan(options: CaseInsensitiveStringMap, schema: StructType) extends Scan with Batch { +class YarnLogsScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder + with Scan with Batch with Serializable { private val appId: String = options.getOrDefault("appId", "*") override def readSchema(): StructType = schema @@ -30,5 +31,8 @@ class YarnLogsScan(options: CaseInsensitiveStringMap, schema: StructType) extend Array(new YarnLogsPartition(appId)) } - override def createReaderFactory(): PartitionReaderFactory = ??? + override def createReaderFactory(): PartitionReaderFactory = + new YarnLogsReaderFactory + + override def build(): Scan = this } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala index 4ed4e7f5341..12d9b6413f9 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kyuubi.spark.connector.yarn import java.util @@ -22,21 +21,25 @@ import java.util import scala.jdk.CollectionConverters.setAsJavaSetConverter import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} -import org.apache.spark.sql.connector.read.ScanBuilder -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class YarnLogsTable extends Table with SupportsRead { - override def name(): String = "agg_logs" + override def name(): String = "app_log" override def schema(): StructType = new StructType(Array( StructField("appId", StringType, nullable = false), - StructField("logLevel", StringType, nullable = false), + StructField("user", StringType, nullable = false), + StructField("rowIndex", IntegerType, nullable = false), StructField("message", StringType, nullable = true))) override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava - override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = ??? + override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): Scan = + new YarnLogsScan( + caseInsensitiveStringMap, + schema()) } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala new file mode 100644 index 00000000000..e07dce5aad5 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala @@ -0,0 +1,63 @@ +/* + * 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.spark.connector.yarn + +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.yarn.client.api.YarnClient + +class SparkYarnConnectorOnYarn extends WithKyuubiServerAndYarnMiniCluster { + override def beforeAll(): Unit = { + super.beforeAll() + // init log dir and set permission + val fs = FileSystem.get(hdfsConf) + val logDir = new Path("/tmp/logs") + fs.mkdirs(logDir) + fs.setPermission(logDir, new org.apache.hadoop.fs.permission.FsPermission("777")) + info(s"hdfs web address: http://${fs.getConf.get("dfs.http.address")}") + fs.close() + // mock app submit + for (i <- 1 to 10) { + submitMockTaskOnYarn() + } + } + + test("yarn - list applications") { + val yarnClient = YarnClient.createYarnClient() + yarnClient.init(yarnConf) + yarnClient.start() + val applications = yarnClient.getApplications + applications.forEach(app => { + val applicationId = app.getApplicationId + val applicationType = app.getApplicationType + val applicationName = app.getName + val applicationState = app.getYarnApplicationState + val currentApplicationAttemptId = app.getCurrentApplicationAttemptId + val startTime = app.getStartTime + val finishTime = app.getFinishTime + val submitTime = app.getSubmitTime + val launchTime = app.getLaunchTime + info(s"get application info from client, ${applicationId}, " + + s"submit: ${submitTime}" + + s"launch: ${launchTime}" + + s"start: ${startTime}" + + s"finish: ${finishTime}" + + s"${applicationName}, ${applicationType}, ${applicationState.name()}") + }) + yarnClient.close() + } +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala new file mode 100644 index 00000000000..675d6f6c92b --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -0,0 +1,205 @@ +/* + * 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.spark.connector.yarn + +import java.io.{File, FileWriter} +import java.util.Collections + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.records.{ApplicationSubmissionContext, ContainerLaunchContext, Resource, YarnApplicationState} +import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.Records + +import org.apache.kyuubi.{KyuubiFunSuite, Utils, WithKyuubiServer} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf.KYUUBI_ENGINE_ENV_PREFIX +import org.apache.kyuubi.server.{MiniDFSService, MiniYarnService} +import org.apache.kyuubi.util.JavaUtils + +trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiServer { + + override protected val conf: KyuubiConf = new KyuubiConf(false) + + val kyuubiHome: String = JavaUtils.getCodeSourceLocation(getClass).split("extensions").head + + protected var miniHdfsService: MiniDFSService = _ + + protected var miniYarnService: MiniYarnService = _ + + protected var hdfsServiceUrl: String = _ + + protected var hdfsConf: Configuration = _ + + protected val yarnConf: YarnConfiguration = { + val yarnConfig = new YarnConfiguration() + + // configurations copied from org.apache.flink.yarn.YarnTestBase + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 32) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096) + + yarnConfig.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true) + yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2) + yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4) + yarnConfig.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600) + yarnConfig.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false) + // memory is overwritten in the MiniYARNCluster. + // so we have to change the number of cores for testing. + yarnConfig.setInt(YarnConfiguration.NM_VCORES, 666) + yarnConfig.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 1000) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, 5000) + + // capacity-scheduler.xml is missing in hadoop-client-minicluster so this is a workaround + yarnConfig.set("yarn.scheduler.capacity.root.queues", "default,four_cores_queue") + + yarnConfig.setInt("yarn.scheduler.capacity.root.default.capacity", 100) + yarnConfig.setFloat("yarn.scheduler.capacity.root.default.user-limit-factor", 1) + yarnConfig.setInt("yarn.scheduler.capacity.root.default.maximum-capacity", 100) + yarnConfig.set("yarn.scheduler.capacity.root.default.state", "RUNNING") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-capacity", 100) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-applications", 10) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-allocation-vcores", 4) + yarnConfig.setFloat("yarn.scheduler.capacity.root.four_cores_queue.user-limit-factor", 1) + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.node-locality-delay", -1) + // Set bind host to localhost to avoid java.net.BindException + yarnConfig.set(YarnConfiguration.RM_BIND_HOST, "localhost") + yarnConfig.set(YarnConfiguration.NM_BIND_HOST, "localhost") + + // Configure YARN log aggregation + yarnConfig.set("yarn.nodemanager.remote-app-log-dir", "/tmp/logs") + yarnConfig.set("yarn.nodemanager.remote-app-log-dir-suffix", "logs") + yarnConfig.set("yarn.log-aggregation-enable", "true") + yarnConfig.set("yarn.log-aggregation.retain-seconds", "3600") + yarnConfig.set("yarn.log-aggregation.retain-check-interval-seconds", "300") + + yarnConfig + } + override def beforeAll(): Unit = { + hdfsConf = new Configuration() + // before HADOOP-18206 (3.4.0), HDFS MetricsLogger strongly depends on + // commons-logging, we should disable it explicitly, otherwise, it throws + // ClassNotFound: org.apache.commons.logging.impl.Log4JLogger + hdfsConf.set("dfs.namenode.metrics.logger.period.seconds", "0") + hdfsConf.set("dfs.datanode.metrics.logger.period.seconds", "0") + + // TODO delete it + hdfsConf.set("hadoop.http.staticuser.user", s"zhangxinsen") + + miniHdfsService = new MiniDFSService(hdfsConf) + miniHdfsService.initialize(conf) + miniHdfsService.start() + + hdfsServiceUrl = s"hdfs://localhost:${miniHdfsService.getDFSPort}" + + // add some hdfs conf in yarn conf + hdfsConf.set("fs.defaultFS", hdfsServiceUrl) + yarnConf.set("fs.defaultFS", hdfsServiceUrl) + yarnConf.addResource(miniHdfsService.getHadoopConf) + + val cp = System.getProperty("java.class.path") + // exclude kyuubi flink engine jar that has SPI for EmbeddedExecutorFactory + // which can't be initialized on the client side + val hadoopJars = cp.split(":").filter(s => !s.contains("flink")) + val hadoopClasspath = hadoopJars.mkString(":") + yarnConf.set("yarn.application.classpath", hadoopClasspath) + + miniYarnService = new MiniYarnService() + miniYarnService.setYarnConf(yarnConf) + miniYarnService.initialize(conf) + miniYarnService.start() + + val hadoopConfDir = Utils.createTempDir().toFile + val writer = new FileWriter(new File(hadoopConfDir, "core-site.xml")) + yarnConf.writeXml(writer) + writer.close() + + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.KYUUBI_HOME", kyuubiHome) + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.HADOOP_CLASSPATH", hadoopClasspath) + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + conf.set(s"flink.containerized.master.env.HADOOP_CLASSPATH", hadoopClasspath) + conf.set(s"flink.containerized.master.env.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + conf.set(s"flink.containerized.taskmanager.env.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + + super.beforeAll() + } + + def submitMockTaskOnYarn(): Unit = { + // Initialize YarnClient + val yarnClient = YarnClient.createYarnClient() + yarnClient.init(yarnConf) + yarnClient.start() + // Create a simple application submission context + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + val applicationId = yarnClient.createApplication() + .getApplicationSubmissionContext.getApplicationId + appContext.setApplicationId(applicationId) + appContext.setApplicationName("TestApp") + + // Set up container launch context (e.g., commands to execute) + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + val commands = Collections.singletonList("echo Hello, MiniYARNCluster! && sleep 2") + amContainer.setCommands(commands) + + // Application Master resource requirements + val capability = Records.newRecord(classOf[Resource]) + capability.setMemorySize(128) + capability.setVirtualCores(1) + + appContext.setResource(capability) + appContext.setAMContainerSpec(amContainer) + + // Submit the application + yarnClient.submitApplication(appContext) + info(s"Application ${applicationId} submitted successfully.") + + // Wait for application to complete + var appReport = yarnClient.getApplicationReport(applicationId) + while (appReport.getYarnApplicationState != YarnApplicationState.FINISHED && + appReport.getYarnApplicationState != YarnApplicationState.FAILED && + appReport.getYarnApplicationState != YarnApplicationState.KILLED) { + info(s"Application State: ${appReport.getYarnApplicationState}") + Thread.sleep(1000) + appReport = yarnClient.getApplicationReport(applicationId) + } + + info(s"Final Application State: ${appReport.getYarnApplicationState}") + + // Clean up + yarnClient.stop() + } + + override def afterAll(): Unit = { + super.afterAll() + if (miniYarnService != null) { + miniYarnService.stop() + miniYarnService = null + } + if (miniHdfsService != null) { + miniHdfsService.stop() + miniHdfsService = null + } + } +} From 827eb91b6ab9412329c943ab058b636e14d0de17 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Wed, 11 Dec 2024 23:58:03 +0800 Subject: [PATCH 03/38] init yarn.default.app table query and impl yarn catalog suite --- .../spark/kyuubi-spark-connector-yarn/pom.xml | 2 +- .../connector/yarn/YarnAppPartition.scala | 22 +++++ .../yarn/YarnAppPartitionReader.scala | 90 +++++++++++++++++++ .../connector/yarn/YarnAppReaderFactory.scala | 29 ++++++ .../spark/connector/yarn/YarnAppScan.scala | 41 +++++++++ .../yarn/YarnApplicationFilter.scala | 31 +++++++ .../connector/yarn/YarnApplicationTable.scala | 52 +++++++++++ .../spark/connector/yarn/YarnCatalog.scala | 9 +- .../spark/connector/yarn/YarnLogsTable.scala | 4 +- ...scala => SparkYarnConnectorWithYarn.scala} | 28 +----- .../connector/yarn/YarnCatalogSuite.scala | 82 +++++++++++++++++ 11 files changed, 355 insertions(+), 35 deletions(-) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppReaderFactory.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/{SparkYarnConnectorOnYarn.scala => SparkYarnConnectorWithYarn.scala} (55%) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml index 6ee869b5aee..8b5e3c43694 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml +++ b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml @@ -129,7 +129,7 @@ org.apache.hadoop hadoop-client-api - test + provided diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala new file mode 100644 index 00000000000..ec5d4d978b2 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read.InputPartition + +class YarnAppPartition(val appId: String) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala new file mode 100644 index 00000000000..5a25ba16ade --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -0,0 +1,90 @@ +/* + * 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.spark.connector.yarn + +import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.connector.read.PartitionReader + +class YarnAppPartitionReader(inputPartition: YarnAppPartition) + extends PartitionReader[InternalRow] { + + private val appIterator = fetchApp(inputPartition).iterator + + override def next(): Boolean = appIterator.hasNext + + override def get(): InternalRow = { + val app = appIterator.next() + new GenericInternalRow(Array[Any]( + app.id, + app.appType, + app.user, + app.name, + app.state, + app.queue, + app.attemptId, + app.submitTime, + app.launchTime, + app.startTime, + app.finishTime)) + } + + override def close(): Unit = {} + + private def fetchApp(inputPartition: YarnAppPartition): Seq[YarnApplication] = { + val yarnClient = YarnClient.createYarnClient() + val yarnConf = new YarnConfiguration() + yarnClient.init(yarnConf) + yarnClient.start() + // fet apps + val applicationReports = yarnClient.getApplications() + val appSeq = Seq[YarnApplication]() + applicationReports.forEach(app => { + appSeq :+ YarnApplication( + id = app.getApplicationType, + appType = app.getApplicationType, + user = app.getUser, + name = app.getName, + state = app.getYarnApplicationState.name, + queue = app.getQueue, + attemptId = app.getCurrentApplicationAttemptId.toString, + submitTime = app.getSubmitTime, + launchTime = app.getLaunchTime, + startTime = app.getStartTime, + finishTime = app.getFinishTime) + }) + yarnClient.close() + appSeq + } +} + +// Helper class to represent app +case class YarnApplication( + id: String, + appType: String, + user: String, + name: String, + state: String, + queue: String, + attemptId: String, + submitTime: Long, + launchTime: Long, + startTime: Long, + finishTime: Long) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppReaderFactory.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppReaderFactory.scala new file mode 100644 index 00000000000..edb9cbf646a --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppReaderFactory.scala @@ -0,0 +1,29 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} + +class YarnAppReaderFactory extends PartitionReaderFactory { + + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + val yarnAppPartition = partition.asInstanceOf[YarnAppPartition] + new YarnAppPartitionReader(yarnAppPartition) + } +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala new file mode 100644 index 00000000000..7c4fc8e6cb3 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -0,0 +1,41 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder + with Scan with Batch with Serializable { + private val appId: String = options.getOrDefault("appId", "*") + // private val yarnApplication: YarnApplication = new YarnApplication(id = + // options.get("appId").isEmpty + // ) + override def readSchema(): StructType = schema + + override def planInputPartitions(): Array[InputPartition] = { + // Fetch app for the given appId (filtering logic can be added) + Array(new YarnLogsPartition(appId)) + } + + override def createReaderFactory(): PartitionReaderFactory = + new YarnLogsReaderFactory + + override def build(): Scan = this +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala new file mode 100644 index 00000000000..943db37d649 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala @@ -0,0 +1,31 @@ +/* + * 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.spark.connector.yarn + +case class YarnApplicationFilter( + id: Option[String], + appType: Option[String], + user: Option[String], + name: Option[String], + state: Option[String], + queue: Option[String], + attemptId: Option[String], + submitTime: Option[Long], + launchTime: Option[Long], + startTime: Option[Long], + finishTime: Option[Long]) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala new file mode 100644 index 00000000000..01e68eb6a87 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala @@ -0,0 +1,52 @@ +/* + * 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.spark.connector.yarn + +import java.util + +import scala.jdk.CollectionConverters.setAsJavaSetConverter + +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class YarnApplicationTable extends Table with SupportsRead { + override def name(): String = "apps" + + override def schema(): StructType = + new StructType(Array( + StructField("id", StringType, nullable = false), + StructField("type", StringType, nullable = false), + StructField("user", StringType, nullable = false), + StructField("name", StringType, nullable = false), + StructField("state", StringType, nullable = false), + StructField("queue", StringType, nullable = false), + StructField("attempt_id", StringType, nullable = false), + StructField("submit_time", LongType, nullable = false), + StructField("launch_time", LongType, nullable = false), + StructField("start_time", LongType, nullable = false), + StructField("finish_time", LongType, nullable = false))) + + override def capabilities(): util.Set[TableCapability] = + Set(TableCapability.BATCH_READ).asJava + + override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = + new YarnAppScan( + caseInsensitiveStringMap, + schema()) +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 29dd04db89f..831640c6a25 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -22,7 +22,7 @@ import java.util import scala.jdk.CollectionConverters.mapAsJavaMapConverter import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -43,9 +43,8 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { override def loadTable(identifier: Identifier): Table = identifier.name match { case "app_logs" => new YarnLogsTable - // TODO impl YarnAppTable - // case "apps" => new YarnAppTable - case _ => throw new UnsupportedOperationException(s"Table ${identifier.name()} not found") + case "apps" => new YarnApplicationTable + case _ => throw new NoSuchTableException(s"${identifier.name}") } @@ -70,7 +69,7 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { } override def listNamespaces(): Array[Array[String]] = { - Array("default") + Array(Array("default")) } override def listNamespaces(namespace: Array[String]): Array[Array[String]] = namespace match { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala index 12d9b6413f9..3d30793237e 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala @@ -21,7 +21,7 @@ import java.util import scala.jdk.CollectionConverters.setAsJavaSetConverter import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,7 +38,7 @@ class YarnLogsTable extends Table with SupportsRead { override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava - override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): Scan = + override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = new YarnLogsScan( caseInsensitiveStringMap, schema()) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala similarity index 55% rename from extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index e07dce5aad5..20f1a5d6d76 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorOnYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -18,9 +18,8 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.yarn.client.api.YarnClient -class SparkYarnConnectorOnYarn extends WithKyuubiServerAndYarnMiniCluster { +trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { override def beforeAll(): Unit = { super.beforeAll() // init log dir and set permission @@ -35,29 +34,4 @@ class SparkYarnConnectorOnYarn extends WithKyuubiServerAndYarnMiniCluster { submitMockTaskOnYarn() } } - - test("yarn - list applications") { - val yarnClient = YarnClient.createYarnClient() - yarnClient.init(yarnConf) - yarnClient.start() - val applications = yarnClient.getApplications - applications.forEach(app => { - val applicationId = app.getApplicationId - val applicationType = app.getApplicationType - val applicationName = app.getName - val applicationState = app.getYarnApplicationState - val currentApplicationAttemptId = app.getCurrentApplicationAttemptId - val startTime = app.getStartTime - val finishTime = app.getFinishTime - val submitTime = app.getSubmitTime - val launchTime = app.getLaunchTime - info(s"get application info from client, ${applicationId}, " + - s"submit: ${submitTime}" + - s"launch: ${launchTime}" + - s"start: ${startTime}" + - s"finish: ${finishTime}" + - s"${applicationName}, ${applicationType}, ${applicationState.name()}") - }) - yarnClient.close() - } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala new file mode 100644 index 00000000000..73303d764be --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession + +class YarnCatalogSuite extends SparkYarnConnectorWithYarn { + test("get catalog name") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { _ => + val catalog = new YarnCatalog + val catalogName = "yarn" + catalog.initialize(catalogName, CaseInsensitiveStringMap.empty()) + assert(catalog.name() == catalogName) + } + } + + test("supports namespaces") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + // spark.sql("show tables").collect().head.get(1) + assert(spark.sql("SHOW DATABASES").collect().length == 1) + assert(spark.sql("SHOW NAMESPACES").collect().length == 1) + assert(spark.sql("SHOW DATABASES").collect().head.get(0) == "default") + } + } + + test("show tables") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + assert(spark.sql("SHOW TABLES").collect().length == 2) + } + } + + test("nonexistent table") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + val exception = intercept[AnalysisException] { + spark.table("yarn.default.nonexistent_table") + } + assert(exception.message.contains("Table or view not found") + || exception.message.contains("TABLE_OR_VIEW_NOT_FOUND")) + } + } +} From a86b79c72e79d52c218fc2c360ce68cefeeee51b Mon Sep 17 00:00:00 2001 From: native-zhang Date: Thu, 12 Dec 2024 00:09:43 +0800 Subject: [PATCH 04/38] make YarnAppScan support BATCH_SCAN --- .../spark/connector/yarn/YarnAppScan.scala | 3 ++ .../spark/connector/yarn/YarnQuerySuite.scala | 37 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 7c4fc8e6cb3..8bd7540811d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -23,6 +23,9 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder with Scan with Batch with Serializable { + + override def toBatch: Batch = this + private val appId: String = options.getOrDefault("appId", "*") // private val yarnApplication: YarnApplication = new YarnApplication(id = // options.get("appId").isEmpty diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala new file mode 100644 index 00000000000..5591d1451f7 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -0,0 +1,37 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession + +class YarnQuerySuiteextends extends SparkYarnConnectorWithYarn { + test("get app tables") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + assert(spark.sql("SHOW TABLES").collect().length == 2) + } + } +} From d03e568e3c63c3add7d8a97004d9aec7bdba0c6a Mon Sep 17 00:00:00 2001 From: naive-zhang Date: Thu, 12 Dec 2024 18:36:18 +0800 Subject: [PATCH 05/38] modify reader logic in YarnAppScan --- .../connector/yarn/YarnAppPartitionReader.scala | 15 ++++++++------- .../kyuubi/spark/connector/yarn/YarnAppScan.scala | 5 +++-- .../kyuubi/spark/connector/yarn/YarnCatalog.scala | 1 - .../spark/connector/yarn/YarnQuerySuite.scala | 3 ++- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 5a25ba16ade..4fdb700eb94 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.unsafe.types.UTF8String class YarnAppPartitionReader(inputPartition: YarnAppPartition) extends PartitionReader[InternalRow] { @@ -33,13 +34,13 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) override def get(): InternalRow = { val app = appIterator.next() new GenericInternalRow(Array[Any]( - app.id, - app.appType, - app.user, - app.name, - app.state, - app.queue, - app.attemptId, + UTF8String.fromString(app.id), + UTF8String.fromString(app.appType), + UTF8String.fromString(app.user), + UTF8String.fromString(app.name), + UTF8String.fromString(app.state), + UTF8String.fromString(app.queue), + UTF8String.fromString(app.attemptId), app.submitTime, app.launchTime, app.startTime, diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 8bd7540811d..628e1f087cf 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -27,6 +27,7 @@ class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends override def toBatch: Batch = this private val appId: String = options.getOrDefault("appId", "*") + private val _options: CaseInsensitiveStringMap = options // private val yarnApplication: YarnApplication = new YarnApplication(id = // options.get("appId").isEmpty // ) @@ -34,11 +35,11 @@ class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends override def planInputPartitions(): Array[InputPartition] = { // Fetch app for the given appId (filtering logic can be added) - Array(new YarnLogsPartition(appId)) + Array(new YarnAppPartition(appId)) } override def createReaderFactory(): PartitionReaderFactory = - new YarnLogsReaderFactory + new YarnAppReaderFactory override def build(): Scan = this } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 831640c6a25..36cb65ee41a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -74,7 +74,6 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { override def listNamespaces(namespace: Array[String]): Array[Array[String]] = namespace match { case Array() => listNamespaces() - // TODO make it available case Array(db) if db eq "default" => listNamespaces() case _ => throw new NoSuchNamespaceException(namespace) } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index 5591d1451f7..dd7844e7ac5 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.SparkSession import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession -class YarnQuerySuiteextends extends SparkYarnConnectorWithYarn { +class YarnQuerySuite extends SparkYarnConnectorWithYarn { test("get app tables") { val sparkConf = new SparkConf() .setMaster("local[*]") @@ -31,6 +31,7 @@ class YarnQuerySuiteextends extends SparkYarnConnectorWithYarn { .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") + val apps = spark.sql("select * from apps").collect() assert(spark.sql("SHOW TABLES").collect().length == 2) } } From 133d30711bd9a0078a99596c637d43f53e75933d Mon Sep 17 00:00:00 2001 From: native-zhang Date: Thu, 12 Dec 2024 23:32:26 +0800 Subject: [PATCH 06/38] get hadoop conf from SparkSession.active.SparkConf --- .../connector/yarn/YarnAppPartition.scala | 3 +- .../yarn/YarnAppPartitionReader.scala | 15 ++++-- .../spark/connector/yarn/YarnAppScan.scala | 51 ++++++++++++++++++- .../yarn/SparkYarnConnectorWithYarn.scala | 31 ++++++++++- .../spark/connector/yarn/YarnQuerySuite.scala | 3 +- 5 files changed, 92 insertions(+), 11 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala index ec5d4d978b2..c3334322173 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -19,4 +19,5 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition -class YarnAppPartition(val appId: String) extends InputPartition +case class YarnAppPartition(val appId: String, val hadoopConfMap: Map[String, String]) + extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 4fdb700eb94..adcf55cf003 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,15 +17,17 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.client.api.YarnClient -import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String + class YarnAppPartitionReader(inputPartition: YarnAppPartition) - extends PartitionReader[InternalRow] { + extends PartitionReader[InternalRow] with Logging { private val appIterator = fetchApp(inputPartition).iterator @@ -50,12 +52,15 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) override def close(): Unit = {} private def fetchApp(inputPartition: YarnAppPartition): Seq[YarnApplication] = { + val hadoopConf = new Configuration() + inputPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) + inputPartition.hadoopConfMap.foreach(kv => logInfo(s"hadoop conf: ${kv._1}: ${kv._2}")) val yarnClient = YarnClient.createYarnClient() - val yarnConf = new YarnConfiguration() - yarnClient.init(yarnConf) + yarnClient.init(hadoopConf) yarnClient.start() + val queues = yarnClient.getAllQueues // fet apps - val applicationReports = yarnClient.getApplications() + val applicationReports = yarnClient.getApplications val appSeq = Seq[YarnApplication]() applicationReports.forEach(app => { appSeq :+ YarnApplication( diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 628e1f087cf..009f38bd21e 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -17,12 +17,17 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.hadoop.conf.Configuration +import org.apache.kyuubi.spark.connector.yarn.YarnAppScan.resolvePlaceholders +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap + class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder with Scan with Batch with Serializable { + private var hadoopConfMap: Map[String, String] = _ override def toBatch: Batch = this @@ -35,11 +40,53 @@ class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends override def planInputPartitions(): Array[InputPartition] = { // Fetch app for the given appId (filtering logic can be added) - Array(new YarnAppPartition(appId)) + // hadoopConf can not be serialized correctly + // use map here + Array(new YarnAppPartition(appId, hadoopConfMap)) } override def createReaderFactory(): PartitionReaderFactory = new YarnAppReaderFactory - override def build(): Scan = this + override def build(): Scan = { + val hadoopConf = SparkSession.active.sparkContext.hadoopConfiguration + hadoopConfMap = resolvePlaceholders(hadoopConf) + this + } +} + +object YarnAppScan { + /** + * resolve Hadoop Configuration ,replace "${xxx}" with the value of xxx + * + * for example + * input Configuration, where key1=value1 and key2=${key1} + * output Map(key1 -> value1, key2 -> value1) + * + * @param inputConf the input Configuration + * @return resolved Map + */ + def resolvePlaceholders(inputConf: Configuration): Map[String, String] = { + import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter + val conf = inputConf.asScala.map(entry => entry.getKey -> entry.getValue).toMap + + def resolve(value: String, visitedKeys: Set[String]): String = { + val placeholderPattern = "\\$\\{([^}]+)\\}".r + + placeholderPattern.replaceAllIn(value, m => { + val key = m.group(1) + if (visitedKeys.contains(key)) { + throw new IllegalArgumentException(s"Circular reference detected for key: $key") + } + conf.get(key) match { + case Some(replacement) => resolve(replacement, visitedKeys + key) + case None => resolve("", visitedKeys + key) + } + }) + } + + conf.map { case (key, value) => + key -> resolve(value, Set(key)) + } + } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index 20f1a5d6d76..55772145b8f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -17,11 +17,30 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import java.io.{File, FileOutputStream} + trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { + def writeConfigToFile(conf: Configuration, filePath: String): Unit = { + val file = new File(filePath) + info(s"xml path: ${file.getAbsolutePath}") + val outputStream = new FileOutputStream(file) + try { + conf.writeXml(outputStream) + } finally { + outputStream.close() + } + } + override def beforeAll(): Unit = { super.beforeAll() + // get all conf and set up hadoop conf dir + if (!new File("tmp/hadoop-conf").exists()) new File("tmp/hadoop-conf").mkdirs() + writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/core-site.xml") + writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/hdfs-site.xml") + writeConfigToFile(miniYarnService.getYarnConf, "tmp/hadoop-conf/yarn-site.xml") // init log dir and set permission val fs = FileSystem.get(hdfsConf) val logDir = new Path("/tmp/logs") @@ -30,8 +49,18 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { info(s"hdfs web address: http://${fs.getConf.get("dfs.http.address")}") fs.close() // mock app submit - for (i <- 1 to 10) { + for (i <- 1 to 3) { submitMockTaskOnYarn() } + // log all conf + miniHdfsService.getHadoopConf.forEach(kv => + info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) + miniYarnService.getYarnConf.forEach(kv => info(s"mini yarn conf ${kv.getKey}: ${kv.getValue}")) + } + + override def afterAll(): Unit = { + super.afterAll() + // delete hadoop conf dir + } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index dd7844e7ac5..236ce51d073 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -17,11 +17,10 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession - class YarnQuerySuite extends SparkYarnConnectorWithYarn { test("get app tables") { val sparkConf = new SparkConf() From db4a7743641c74fa3cb75d8f7cbf412501489cd9 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Fri, 13 Dec 2024 22:38:29 +0800 Subject: [PATCH 07/38] add xml file sense --- .../connector/yarn/YarnAppPartitionReader.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index adcf55cf003..e1f348ff278 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -18,6 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow @@ -53,12 +54,20 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) private def fetchApp(inputPartition: YarnAppPartition): Seq[YarnApplication] = { val hadoopConf = new Configuration() - inputPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) - inputPartition.hadoopConfMap.foreach(kv => logInfo(s"hadoop conf: ${kv._1}: ${kv._2}")) + val confPath = new Path("tmp/hadoop-conf") + val fs = confPath.getFileSystem(hadoopConf) + val fileStatuses = fs.listStatus(confPath) + fileStatuses.foreach( + fileStatus => { + if (fileStatus.isFile && fileStatus.getPath.getName.endsWith(".xml")) { + hadoopConf.addResource(fileStatus.getPath) + } + } + ) + fs.close() val yarnClient = YarnClient.createYarnClient() yarnClient.init(hadoopConf) yarnClient.start() - val queues = yarnClient.getAllQueues // fet apps val applicationReports = yarnClient.getApplications val appSeq = Seq[YarnApplication]() From f6064f1d00b0e27ac87555eaaa079b1da37b3141 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 14 Dec 2024 02:24:34 +0800 Subject: [PATCH 08/38] split hdfs related xml and yarn related xml --- .../connector/yarn/YarnAppPartition.scala | 2 +- .../yarn/YarnAppPartitionReader.scala | 16 +++--- .../spark/connector/yarn/YarnAppScan.scala | 50 ++----------------- .../spark/connector/yarn/YarnCatalog.scala | 13 +++-- .../yarn/SparkYarnConnectorWithYarn.scala | 4 +- .../WithKyuubiServerAndYarnMiniCluster.scala | 25 ++++++++-- .../connector/yarn/YarnCatalogSuite.scala | 23 +++++++-- .../spark/connector/yarn/YarnQuerySuite.scala | 48 +++++++++++++++++- 8 files changed, 111 insertions(+), 70 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala index c3334322173..5bfab6b5140 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -19,5 +19,5 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition -case class YarnAppPartition(val appId: String, val hadoopConfMap: Map[String, String]) +case class YarnAppPartition(val appId: String) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index e1f348ff278..ba01b6d5d33 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -26,6 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + + class YarnAppPartitionReader(inputPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { @@ -52,27 +56,26 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) override def close(): Unit = {} - private def fetchApp(inputPartition: YarnAppPartition): Seq[YarnApplication] = { + private def fetchApp(inputPartition: YarnAppPartition): mutable.Seq[YarnApplication] = { val hadoopConf = new Configuration() val confPath = new Path("tmp/hadoop-conf") val fs = confPath.getFileSystem(hadoopConf) val fileStatuses = fs.listStatus(confPath) fileStatuses.foreach( fileStatus => { - if (fileStatus.isFile && fileStatus.getPath.getName.endsWith(".xml")) { + if (fileStatus.isFile && fileStatus.getPath.getName.endsWith("yarn-site.xml")) { hadoopConf.addResource(fileStatus.getPath) } } ) - fs.close() val yarnClient = YarnClient.createYarnClient() + hadoopConf.get("yarn.resourcemanager.address") yarnClient.init(hadoopConf) yarnClient.start() // fet apps val applicationReports = yarnClient.getApplications - val appSeq = Seq[YarnApplication]() - applicationReports.forEach(app => { - appSeq :+ YarnApplication( + val appSeq = applicationReports.asScala.map(app => { + YarnApplication( id = app.getApplicationType, appType = app.getApplicationType, user = app.getUser, @@ -86,6 +89,7 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) finishTime = app.getFinishTime) }) yarnClient.close() + fs.close() appSeq } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 009f38bd21e..c85821a5548 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -17,17 +17,13 @@ package org.apache.kyuubi.spark.connector.yarn -import org.apache.hadoop.conf.Configuration -import org.apache.kyuubi.spark.connector.yarn.YarnAppScan.resolvePlaceholders -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder +case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder with Scan with Batch with Serializable { - private var hadoopConfMap: Map[String, String] = _ override def toBatch: Batch = this @@ -40,53 +36,15 @@ class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends override def planInputPartitions(): Array[InputPartition] = { // Fetch app for the given appId (filtering logic can be added) - // hadoopConf can not be serialized correctly + // hadoopConf can not be serialized correctly here // use map here - Array(new YarnAppPartition(appId, hadoopConfMap)) + Array(YarnAppPartition(appId)) } override def createReaderFactory(): PartitionReaderFactory = new YarnAppReaderFactory override def build(): Scan = { - val hadoopConf = SparkSession.active.sparkContext.hadoopConfiguration - hadoopConfMap = resolvePlaceholders(hadoopConf) this } -} - -object YarnAppScan { - /** - * resolve Hadoop Configuration ,replace "${xxx}" with the value of xxx - * - * for example - * input Configuration, where key1=value1 and key2=${key1} - * output Map(key1 -> value1, key2 -> value1) - * - * @param inputConf the input Configuration - * @return resolved Map - */ - def resolvePlaceholders(inputConf: Configuration): Map[String, String] = { - import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter - val conf = inputConf.asScala.map(entry => entry.getKey -> entry.getValue).toMap - - def resolve(value: String, visitedKeys: Set[String]): String = { - val placeholderPattern = "\\$\\{([^}]+)\\}".r - - placeholderPattern.replaceAllIn(value, m => { - val key = m.group(1) - if (visitedKeys.contains(key)) { - throw new IllegalArgumentException(s"Circular reference detected for key: $key") - } - conf.get(key) match { - case Some(replacement) => resolve(replacement, visitedKeys + key) - case None => resolve("", visitedKeys + key) - } - }) - } - - conf.map { case (key, value) => - key -> resolve(value, Set(key)) - } - } -} +} \ No newline at end of file diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 36cb65ee41a..52d7fc1e9ad 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -17,10 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import java.util - -import scala.jdk.CollectionConverters.mapAsJavaMapConverter - import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException} import org.apache.spark.sql.connector.catalog._ @@ -28,6 +24,9 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap +import java.util +import scala.jdk.CollectionConverters.mapAsJavaMapConverter + class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { private var catalogName: String = _ @@ -38,7 +37,11 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { } override def listTables(namespace: Array[String]): Array[Identifier] = { - Array(Identifier.of(namespace, "app_logs"), Identifier.of(namespace, "apps")) + namespace(1) match { + case "default" => Array(Identifier.of(namespace, "app_logs"), + Identifier.of(namespace, "apps")) + case _ => throw new NoSuchNamespaceException(namespace) + } } override def loadTable(identifier: Identifier): Table = identifier.name match { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index 55772145b8f..2b72fd478f6 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -49,9 +49,7 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { info(s"hdfs web address: http://${fs.getConf.get("dfs.http.address")}") fs.close() // mock app submit - for (i <- 1 to 3) { - submitMockTaskOnYarn() - } + submitMockTasksInParallelTreeTimes() // log all conf miniHdfsService.getHadoopConf.forEach(kv => info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index 675d6f6c92b..c36f5e1ea95 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -54,8 +54,8 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096) yarnConfig.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true) - yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2) - yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2) + yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1) + yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 10) yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4) yarnConfig.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600) yarnConfig.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false) @@ -160,7 +160,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS // Set up container launch context (e.g., commands to execute) val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - val commands = Collections.singletonList("echo Hello, MiniYARNCluster! && sleep 2") + val commands = Collections.singletonList("echo Hello, MiniYARNCluster! && sleep 5") amContainer.setCommands(commands) // Application Master resource requirements @@ -191,6 +191,25 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS yarnClient.stop() } + def submitMockTasksInParallelTreeTimes(): Unit = { + val threads = (1 to 3).map { i => + new Thread(() => { + info(s"Starting submission in thread $i") + try { + submitMockTaskOnYarn() + } catch { + case e: Exception => + error(s"Error in thread $i: ${e.getMessage}", e) + } + info(s"Finished submission in thread $i") + }) + } + + threads.foreach(_.start()) + threads.foreach(_.join()) + } + + override def afterAll(): Unit = { super.afterAll() if (miniYarnService != null) { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala index 73303d764be..472fec84261 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala @@ -17,11 +17,11 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.util.CaseInsensitiveStringMap - -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} class YarnCatalogSuite extends SparkYarnConnectorWithYarn { test("get catalog name") { @@ -46,7 +46,6 @@ class YarnCatalogSuite extends SparkYarnConnectorWithYarn { .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") - // spark.sql("show tables").collect().head.get(1) assert(spark.sql("SHOW DATABASES").collect().length == 1) assert(spark.sql("SHOW NAMESPACES").collect().length == 1) assert(spark.sql("SHOW DATABASES").collect().head.get(0) == "default") @@ -65,6 +64,22 @@ class YarnCatalogSuite extends SparkYarnConnectorWithYarn { } } + test("nonexistent namespace") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + val namespace = "nonexistent_db" + val exception = intercept[NoSuchNamespaceException] { + spark.sql(s"show tables from yarn.${namespace}.apps") + } + assert(exception.message.contains(s"The schema `${namespace}` cannot be found") + || exception.message.contains("SCHEMA_NOT_FOUND")) + } + } + test("nonexistent table") { val sparkConf = new SparkConf() .setMaster("local[*]") diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index 236ce51d073..346e959c61e 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -17,9 +17,13 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema + +import scala.jdk.CollectionConverters.asScalaBufferConverter class YarnQuerySuite extends SparkYarnConnectorWithYarn { test("get app tables") { @@ -30,8 +34,48 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") - val apps = spark.sql("select * from apps").collect() - assert(spark.sql("SHOW TABLES").collect().length == 2) +// val appCnt = spark.sql("select count(1) from yarn.default.apps") +// .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) +// assert(appCnt >= 0L) + val yarnApplications = spark.sql("select * from yarn.default.apps") + .collect().map(x => x.asInstanceOf[GenericRowWithSchema]) + .map(x => YarnApplication( + id = x.getString(0), + appType = x.getString(1), + user = x.getString(2), + name = x.getString(3), + state = x.getString(4), + queue = x.getString(5), + attemptId = x.getString(6), + submitTime = x.getLong(7), + launchTime = x.getLong(8), + startTime = x.getLong(9), + finishTime = x.getLong(10) + )) + val yarnClient = YarnClient.createYarnClient() + yarnClient.init(yarnConf) + yarnClient.start() + yarnClient.getApplications.forEach( + app => { + val appId = app.getApplicationId.toString + val queryApp = spark.sql(s"select * from yarn.default.apps " + + s"where id = '${appId}'").collect() + val searchedApps = yarnApplications.filter(x => x.id eq appId) + assert(!searchedApps.isEmpty) + val searchedApp = searchedApps.head + assert(searchedApp.appType == app.getApplicationType) + assert(searchedApp.user == app.getUser) + assert(searchedApp.name == app.getName) + assert(searchedApp.state == app.getYarnApplicationState.name) + assert(searchedApp.queue == app.getQueue) + assert(searchedApp.attemptId == app.getCurrentApplicationAttemptId.toString) + assert(searchedApp.submitTime == app.getSubmitTime) + assert(searchedApp.launchTime == app.getLaunchTime) + assert(searchedApp.startTime == app.getStartTime) + assert(searchedApp.finishTime == app.getFinishTime) + } + ) + yarnClient.close() } } } From ea8be0563ee1b14524bf0a89b40901794ae29683 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 14 Dec 2024 02:32:05 +0800 Subject: [PATCH 09/38] reformat app query test code --- .../yarn/YarnAppPartitionReader.scala | 20 +++--- .../spark/connector/yarn/YarnAppScan.scala | 3 +- .../spark/connector/yarn/YarnCatalog.scala | 11 +-- .../yarn/SparkYarnConnectorWithYarn.scala | 4 +- .../WithKyuubiServerAndYarnMiniCluster.scala | 1 - .../connector/yarn/YarnCatalogSuite.scala | 5 +- .../spark/connector/yarn/YarnQuerySuite.scala | 69 +++++++++---------- 7 files changed, 52 insertions(+), 61 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index ba01b6d5d33..7080f9d5a76 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,6 +17,9 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.client.api.YarnClient @@ -26,11 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - - - class YarnAppPartitionReader(inputPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { @@ -61,13 +59,11 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) val confPath = new Path("tmp/hadoop-conf") val fs = confPath.getFileSystem(hadoopConf) val fileStatuses = fs.listStatus(confPath) - fileStatuses.foreach( - fileStatus => { - if (fileStatus.isFile && fileStatus.getPath.getName.endsWith("yarn-site.xml")) { - hadoopConf.addResource(fileStatus.getPath) - } + fileStatuses.foreach(fileStatus => { + if (fileStatus.isFile && fileStatus.getPath.getName.endsWith("yarn-site.xml")) { + hadoopConf.addResource(fileStatus.getPath) } - ) + }) val yarnClient = YarnClient.createYarnClient() hadoopConf.get("yarn.resourcemanager.address") yarnClient.init(hadoopConf) @@ -76,7 +72,7 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) val applicationReports = yarnClient.getApplications val appSeq = applicationReports.asScala.map(app => { YarnApplication( - id = app.getApplicationType, + id = app.getApplicationId.toString, appType = app.getApplicationType, user = app.getUser, name = app.getName, diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index c85821a5548..a78754cf27d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap - case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder with Scan with Batch with Serializable { @@ -47,4 +46,4 @@ case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) ex override def build(): Scan = { this } -} \ No newline at end of file +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 52d7fc1e9ad..24a0969554e 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -17,6 +17,10 @@ package org.apache.kyuubi.spark.connector.yarn +import java.util + +import scala.jdk.CollectionConverters.mapAsJavaMapConverter + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException} import org.apache.spark.sql.connector.catalog._ @@ -24,9 +28,6 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import java.util -import scala.jdk.CollectionConverters.mapAsJavaMapConverter - class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { private var catalogName: String = _ @@ -38,8 +39,8 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { override def listTables(namespace: Array[String]): Array[Identifier] = { namespace(1) match { - case "default" => Array(Identifier.of(namespace, "app_logs"), - Identifier.of(namespace, "apps")) + case "default" => + Array(Identifier.of(namespace, "app_logs"), Identifier.of(namespace, "apps")) case _ => throw new NoSuchNamespaceException(namespace) } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index 2b72fd478f6..2a37deefdf8 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -17,11 +17,11 @@ package org.apache.kyuubi.spark.connector.yarn +import java.io.{File, FileOutputStream} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import java.io.{File, FileOutputStream} - trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { def writeConfigToFile(conf: Configuration, filePath: String): Unit = { val file = new File(filePath) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index c36f5e1ea95..a240dbdf64b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -209,7 +209,6 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS threads.foreach(_.join()) } - override def afterAll(): Unit = { super.afterAll() if (miniYarnService != null) { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala index 472fec84261..0a97223e8e5 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalogSuite.scala @@ -17,11 +17,12 @@ package org.apache.kyuubi.spark.connector.yarn -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.sql.{AnalysisException, SparkSession} + +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession class YarnCatalogSuite extends SparkYarnConnectorWithYarn { test("get catalog name") { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index 346e959c61e..83ec0f018f5 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -18,12 +18,11 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.hadoop.yarn.client.api.YarnClient -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import scala.jdk.CollectionConverters.asScalaBufferConverter +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession class YarnQuerySuite extends SparkYarnConnectorWithYarn { test("get app tables") { @@ -37,44 +36,40 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { // val appCnt = spark.sql("select count(1) from yarn.default.apps") // .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) // assert(appCnt >= 0L) - val yarnApplications = spark.sql("select * from yarn.default.apps") - .collect().map(x => x.asInstanceOf[GenericRowWithSchema]) - .map(x => YarnApplication( - id = x.getString(0), - appType = x.getString(1), - user = x.getString(2), - name = x.getString(3), - state = x.getString(4), - queue = x.getString(5), - attemptId = x.getString(6), - submitTime = x.getLong(7), - launchTime = x.getLong(8), - startTime = x.getLong(9), - finishTime = x.getLong(10) - )) val yarnClient = YarnClient.createYarnClient() yarnClient.init(yarnConf) yarnClient.start() - yarnClient.getApplications.forEach( - app => { - val appId = app.getApplicationId.toString - val queryApp = spark.sql(s"select * from yarn.default.apps " + - s"where id = '${appId}'").collect() - val searchedApps = yarnApplications.filter(x => x.id eq appId) - assert(!searchedApps.isEmpty) - val searchedApp = searchedApps.head - assert(searchedApp.appType == app.getApplicationType) - assert(searchedApp.user == app.getUser) - assert(searchedApp.name == app.getName) - assert(searchedApp.state == app.getYarnApplicationState.name) - assert(searchedApp.queue == app.getQueue) - assert(searchedApp.attemptId == app.getCurrentApplicationAttemptId.toString) - assert(searchedApp.submitTime == app.getSubmitTime) - assert(searchedApp.launchTime == app.getLaunchTime) - assert(searchedApp.startTime == app.getStartTime) - assert(searchedApp.finishTime == app.getFinishTime) - } - ) + yarnClient.getApplications.forEach(app => { + val appId = app.getApplicationId.toString + val queryApps = spark.sql(s"select * from yarn.default.apps " + + s"where id = '${appId}'").collect() + .map(x => x.asInstanceOf[GenericRowWithSchema]) + .map(x => + YarnApplication( + id = x.getString(0), + appType = x.getString(1), + user = x.getString(2), + name = x.getString(3), + state = x.getString(4), + queue = x.getString(5), + attemptId = x.getString(6), + submitTime = x.getLong(7), + launchTime = x.getLong(8), + startTime = x.getLong(9), + finishTime = x.getLong(10))) + assert(!queryApps.isEmpty) + val queryApp = queryApps.head + assert(queryApp.appType == app.getApplicationType) + assert(queryApp.user == app.getUser) + assert(queryApp.name == app.getName) + assert(queryApp.state == app.getYarnApplicationState.name) + assert(queryApp.queue == app.getQueue) + assert(queryApp.attemptId == app.getCurrentApplicationAttemptId.toString) + assert(queryApp.submitTime == app.getSubmitTime) + assert(queryApp.launchTime == app.getLaunchTime) + assert(queryApp.startTime == app.getStartTime) + assert(queryApp.finishTime == app.getFinishTime) + }) yarnClient.close() } } From e092859fdb49803f1c66b672a01cf5cb9a5c9a89 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 14 Dec 2024 17:39:47 +0800 Subject: [PATCH 10/38] try to read from hdfs --- .../connector/yarn/YarnAppPartition.scala | 2 +- .../yarn/YarnAppPartitionReader.scala | 46 +++++++++++++----- .../spark/connector/yarn/YarnAppScan.scala | 5 +- .../yarn/YarnApplicationFilter.scala | 31 ------------ .../yarn/SparkYarnConnectorWithYarn.scala | 4 +- .../spark/connector/yarn/YarnQuerySuite.scala | 48 ++++++++++++++++--- 6 files changed, 84 insertions(+), 52 deletions(-) delete mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala index 5bfab6b5140..18cff34dff2 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -19,5 +19,5 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition -case class YarnAppPartition(val appId: String) +case class YarnAppPartition(yarnConfDir: Option[String], hdfsConfDir: Option[String]) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 7080f9d5a76..a6137e6d35c 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,9 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.client.api.YarnClient @@ -29,9 +26,20 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + class YarnAppPartitionReader(inputPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { + private val YARN_XML_FILENAME: String = "yarn-site.xml" + + private val CORE_XML_FILENAME: String = "core-site.xml" + + private val HDFS_XML_FILENAME: String = "hdfs-site.xml" + + private val CONF_DIR_PROP_KEY: String = "HADOOP_CONF_DIR" + private val appIterator = fetchApp(inputPartition).iterator override def next(): Boolean = appIterator.hasNext @@ -56,16 +64,31 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) private def fetchApp(inputPartition: YarnAppPartition): mutable.Seq[YarnApplication] = { val hadoopConf = new Configuration() - val confPath = new Path("tmp/hadoop-conf") + val confPath = new Path(inputPartition.yarnConfDir.getOrElse( + sys.props.getOrElse(CONF_DIR_PROP_KEY, + sys.env.getOrElse(CONF_DIR_PROP_KEY, + "./")) + )) + if (confPath.toString.startsWith("hdfs") && inputPartition.hdfsConfDir.isDefined) { + // load core-site.xml and hdfs-site.xml + val coreSitePath = new Path("file://" + inputPartition.hdfsConfDir.get + "/core-site.xml") + val hdfsSitePath = new Path("file://" + inputPartition.hdfsConfDir.get + "/hdfs-site.xml") + hadoopConf.addResource(coreSitePath) + hadoopConf.addResource(hdfsSitePath) + } val fs = confPath.getFileSystem(hadoopConf) - val fileStatuses = fs.listStatus(confPath) - fileStatuses.foreach(fileStatus => { - if (fileStatus.isFile && fileStatus.getPath.getName.endsWith("yarn-site.xml")) { - hadoopConf.addResource(fileStatus.getPath) - } - }) + try { + val fileStatuses = fs.listStatus(confPath) + fileStatuses.foreach(fileStatus => { + if (fileStatus.isFile && fileStatus.getPath.getName.endsWith(YARN_XML_FILENAME)) { + hadoopConf.addResource(fileStatus.getPath) + } + }) + } + finally { + fs.close() + } val yarnClient = YarnClient.createYarnClient() - hadoopConf.get("yarn.resourcemanager.address") yarnClient.init(hadoopConf) yarnClient.start() // fet apps @@ -85,7 +108,6 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) finishTime = app.getFinishTime) }) yarnClient.close() - fs.close() appSeq } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index a78754cf27d..363fce9e928 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -17,6 +17,7 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -37,7 +38,9 @@ case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) ex // Fetch app for the given appId (filtering logic can be added) // hadoopConf can not be serialized correctly here // use map here - Array(YarnAppPartition(appId)) + val yarnConfDir = SparkSession.active.conf.getOption("spark.sql.catalog.yarn.dir.conf.yarn") + val hdfsConfDir = SparkSession.active.conf.getOption("spark.sql.catalog.yarn.dir.conf.hdfs") + Array(new YarnAppPartition(yarnConfDir, hdfsConfDir)) } override def createReaderFactory(): PartitionReaderFactory = diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala deleted file mode 100644 index 943db37d649..00000000000 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationFilter.scala +++ /dev/null @@ -1,31 +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.kyuubi.spark.connector.yarn - -case class YarnApplicationFilter( - id: Option[String], - appType: Option[String], - user: Option[String], - name: Option[String], - state: Option[String], - queue: Option[String], - attemptId: Option[String], - submitTime: Option[Long], - launchTime: Option[Long], - startTime: Option[Long], - finishTime: Option[Long]) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index 2a37deefdf8..47ae7adbc33 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -40,7 +40,9 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { if (!new File("tmp/hadoop-conf").exists()) new File("tmp/hadoop-conf").mkdirs() writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/core-site.xml") writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/hdfs-site.xml") - writeConfigToFile(miniYarnService.getYarnConf, "tmp/hadoop-conf/yarn-site.xml") + if (!new File("tmp/hdfs-conf").exists()) new File("tmp/hdfs-conf").mkdirs() + writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hdfs-conf/core-site.xml") + writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hdfs-conf/hdfs-site.xml") // init log dir and set permission val fs = FileSystem.get(hdfsConf) val logDir = new Path("/tmp/logs") diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index 83ec0f018f5..24fc06312ca 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -17,25 +17,26 @@ package org.apache.kyuubi.spark.connector.yarn +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession - class YarnQuerySuite extends SparkYarnConnectorWithYarn { - test("get app tables") { + test("query for table apps") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") .set("spark.sql.catalogImplementation", "in-memory") .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + .set("spark.sql.catalog.yarn.dir.conf.yarn", "tmp/hadoop-conf") withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") -// val appCnt = spark.sql("select count(1) from yarn.default.apps") -// .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) -// assert(appCnt >= 0L) + val appCnt = spark.sql("select count(1) from yarn.default.apps") + .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) + assert(appCnt >= 0L) val yarnClient = YarnClient.createYarnClient() yarnClient.init(yarnConf) yarnClient.start() @@ -73,4 +74,39 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { yarnClient.close() } } + + test("use yarn conf with HADOOP_CONF") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + sys.props("HADOOP_CONF_DIR") = "tmp/hadoop-conf" + spark.sql("USE yarn") + val appCnt = spark.sql("select count(1) from yarn.default.apps") + .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) + assert(appCnt >= 0L) + } + } + + test("use yarn conf in hdfs") { + val hadoopConf = new Configuration() + val fs = org.apache.hadoop.fs.FileSystem.get(hadoopConf) + val hadoopConfDir = hadoopConf.get("hadoop.conf.dir") + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + .set("spark.sql.catalog.yarn.dir.conf.yarn", "hdfs:///tmp/hadoop-conf") + // make hdfs available + .set("spark.sql.catalog.yarn.dir.conf.hdfs", "tmp/hadoop-conf") + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + val appCnt = spark.sql("select count(1) from yarn.default.apps") + .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) + assert(appCnt >= 0L) + } + } } From 357c0effc5ac16dc1584007dbd4425bba7dc4513 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 14 Dec 2024 17:57:33 +0800 Subject: [PATCH 11/38] use more elegant method for hdfs and yarn api --- .../connector/yarn/YarnAppPartition.scala | 3 +- .../yarn/YarnAppPartitionReader.scala | 26 +------------ .../spark/connector/yarn/YarnAppScan.scala | 9 +++-- .../spark/connector/yarn/YarnQuerySuite.scala | 39 ++++++++++++++----- 4 files changed, 39 insertions(+), 38 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala index 18cff34dff2..95298ca0c16 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -19,5 +19,6 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition -case class YarnAppPartition(yarnConfDir: Option[String], hdfsConfDir: Option[String]) + +case class YarnAppPartition(hadoopConfMap: Map[String, String]) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index a6137e6d35c..d49a8a551d3 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -18,7 +18,6 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow @@ -64,30 +63,7 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) private def fetchApp(inputPartition: YarnAppPartition): mutable.Seq[YarnApplication] = { val hadoopConf = new Configuration() - val confPath = new Path(inputPartition.yarnConfDir.getOrElse( - sys.props.getOrElse(CONF_DIR_PROP_KEY, - sys.env.getOrElse(CONF_DIR_PROP_KEY, - "./")) - )) - if (confPath.toString.startsWith("hdfs") && inputPartition.hdfsConfDir.isDefined) { - // load core-site.xml and hdfs-site.xml - val coreSitePath = new Path("file://" + inputPartition.hdfsConfDir.get + "/core-site.xml") - val hdfsSitePath = new Path("file://" + inputPartition.hdfsConfDir.get + "/hdfs-site.xml") - hadoopConf.addResource(coreSitePath) - hadoopConf.addResource(hdfsSitePath) - } - val fs = confPath.getFileSystem(hadoopConf) - try { - val fileStatuses = fs.listStatus(confPath) - fileStatuses.foreach(fileStatus => { - if (fileStatus.isFile && fileStatus.getPath.getName.endsWith(YARN_XML_FILENAME)) { - hadoopConf.addResource(fileStatus.getPath) - } - }) - } - finally { - fs.close() - } + inputPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) val yarnClient = YarnClient.createYarnClient() yarnClient.init(hadoopConf) yarnClient.start() diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 363fce9e928..9bc637964d1 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -22,6 +22,8 @@ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap +import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter + case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder with Scan with Batch with Serializable { @@ -38,9 +40,10 @@ case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) ex // Fetch app for the given appId (filtering logic can be added) // hadoopConf can not be serialized correctly here // use map here - val yarnConfDir = SparkSession.active.conf.getOption("spark.sql.catalog.yarn.dir.conf.yarn") - val hdfsConfDir = SparkSession.active.conf.getOption("spark.sql.catalog.yarn.dir.conf.hdfs") - Array(new YarnAppPartition(yarnConfDir, hdfsConfDir)) + Array(new YarnAppPartition( + SparkSession.active.sparkContext + .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap + )) } override def createReaderFactory(): PartitionReaderFactory = diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala index 24fc06312ca..44554ee3e36 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala @@ -17,7 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf @@ -31,7 +30,16 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { .set("spark.ui.enabled", "false") .set("spark.sql.catalogImplementation", "in-memory") .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) - .set("spark.sql.catalog.yarn.dir.conf.yarn", "tmp/hadoop-conf") + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") val appCnt = spark.sql("select count(1) from yarn.default.apps") @@ -81,8 +89,17 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { .set("spark.ui.enabled", "false") .set("spark.sql.catalogImplementation", "in-memory") .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - sys.props("HADOOP_CONF_DIR") = "tmp/hadoop-conf" spark.sql("USE yarn") val appCnt = spark.sql("select count(1) from yarn.default.apps") .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) @@ -91,17 +108,21 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { } test("use yarn conf in hdfs") { - val hadoopConf = new Configuration() - val fs = org.apache.hadoop.fs.FileSystem.get(hadoopConf) - val hadoopConfDir = hadoopConf.get("hadoop.conf.dir") val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") .set("spark.sql.catalogImplementation", "in-memory") .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) - .set("spark.sql.catalog.yarn.dir.conf.yarn", "hdfs:///tmp/hadoop-conf") - // make hdfs available - .set("spark.sql.catalog.yarn.dir.conf.hdfs", "tmp/hadoop-conf") + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") val appCnt = spark.sql("select count(1) from yarn.default.apps") From d8f81843cce7bc19dbfede3c876ba89c5980238c Mon Sep 17 00:00:00 2001 From: native-zhang Date: Tue, 17 Dec 2024 22:44:08 +0800 Subject: [PATCH 12/38] predicates push down in app tables with equalTo(appId) and equalTo(state) --- .../connector/yarn/YarnAppPartition.scala | 4 +- .../yarn/YarnAppPartitionReader.scala | 66 ++++++++++----- .../spark/connector/yarn/YarnAppScan.scala | 26 +++--- .../connector/yarn/YarnAppScanBuilder.scala | 46 ++++++++++ .../connector/yarn/YarnApplicationTable.scala | 6 +- .../yarn/SparkYarnConnectorWithYarn.scala | 32 +------ ...erySuite.scala => YarnAppQuerySuite.scala} | 83 ++++++++++++------- 7 files changed, 163 insertions(+), 100 deletions(-) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/{YarnQuerySuite.scala => YarnAppQuerySuite.scala} (79%) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala index 95298ca0c16..8c806eb9c28 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartition.scala @@ -18,7 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.sources.Filter - -case class YarnAppPartition(hadoopConfMap: Map[String, String]) +case class YarnAppPartition(hadoopConfMap: Map[String, String], filters: Array[Filter]) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index d49a8a551d3..4db1b4f8e2a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,29 +17,24 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.sql.sources.{EqualTo, In} import org.apache.spark.unsafe.types.UTF8String -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - -class YarnAppPartitionReader(inputPartition: YarnAppPartition) +class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { - private val YARN_XML_FILENAME: String = "yarn-site.xml" - - private val CORE_XML_FILENAME: String = "core-site.xml" - - private val HDFS_XML_FILENAME: String = "hdfs-site.xml" - - private val CONF_DIR_PROP_KEY: String = "HADOOP_CONF_DIR" - - private val appIterator = fetchApp(inputPartition).iterator + private val appIterator = fetchApp().iterator override def next(): Boolean = appIterator.hasNext @@ -56,19 +51,48 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) app.submitTime, app.launchTime, app.startTime, - app.finishTime)) + app.finishTime, + UTF8String.fromString(app.trackingUrl), + UTF8String.fromString(app.originalTrackingUrl))) } override def close(): Unit = {} - private def fetchApp(inputPartition: YarnAppPartition): mutable.Seq[YarnApplication] = { + private def fetchApp(): mutable.Seq[YarnApplication] = { val hadoopConf = new Configuration() - inputPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) + yarnAppPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) val yarnClient = YarnClient.createYarnClient() yarnClient.init(hadoopConf) yarnClient.start() // fet apps - val applicationReports = yarnClient.getApplications + val applicationReports: java.util.List[ApplicationReport] = + yarnAppPartition.filters match { + case filters if filters.isEmpty => yarnClient.getApplications + // id => point query + // state => batch query + // type => in (a,b,c), batch query + case filters => + filters.collectFirst { + case EqualTo("id", appId: String) => java.util.Collections.singletonList( + yarnClient.getApplicationReport(ApplicationId.fromString(appId))) + case EqualTo("state", state: String) => + yarnClient.getApplications(java.util.EnumSet.of(YarnApplicationState.valueOf(state))) + case EqualTo("type", appType: String) => + yarnClient.getApplications(java.util.Collections.singleton(appType)) + case In("state", states: Array[Any]) => yarnClient.getApplications( + java.util.EnumSet.copyOf(states.map(x => + YarnApplicationState.valueOf(x.toString)).toList.asJava)) + case In("type", types: Array[Any]) => yarnClient.getApplications( + types.map(x => x.toString).toSet.asJava) + case _ => yarnClient.getApplications() + }.get + } + + // case yarnAppPartition.filters match { + // case filters if filters.isEmpty => yarnClient.getApplications + // case filters => + // yarnClient.getApplications + // } val appSeq = applicationReports.asScala.map(app => { YarnApplication( id = app.getApplicationId.toString, @@ -81,7 +105,9 @@ class YarnAppPartitionReader(inputPartition: YarnAppPartition) submitTime = app.getSubmitTime, launchTime = app.getLaunchTime, startTime = app.getStartTime, - finishTime = app.getFinishTime) + finishTime = app.getFinishTime, + trackingUrl = app.getTrackingUrl, + originalTrackingUrl = app.getOriginalTrackingUrl) }) yarnClient.close() appSeq @@ -100,4 +126,6 @@ case class YarnApplication( submitTime: Long, launchTime: Long, startTime: Long, - finishTime: Long) + finishTime: Long, + trackingUrl: String, + originalTrackingUrl: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 9bc637964d1..6a68138f591 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -17,33 +17,31 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter + import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter - -case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder +case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType, pushed: Array[Filter]) + extends ScanBuilder with Scan with Batch with Serializable { override def toBatch: Batch = this - private val appId: String = options.getOrDefault("appId", "*") - private val _options: CaseInsensitiveStringMap = options - // private val yarnApplication: YarnApplication = new YarnApplication(id = - // options.get("appId").isEmpty - // ) override def readSchema(): StructType = schema override def planInputPartitions(): Array[InputPartition] = { - // Fetch app for the given appId (filtering logic can be added) - // hadoopConf can not be serialized correctly here - // use map here - Array(new YarnAppPartition( + // show pushed + // scalastyle:off println + println(s"Applying filters: ${pushed.mkString(", ")}") + // scalastyle:on println + Array(YarnAppPartition( SparkSession.active.sparkContext - .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap - )) + .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap, + pushed)) } override def createReaderFactory(): PartitionReaderFactory = diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala new file mode 100644 index 00000000000..b0fc0c391a2 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.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.spark.connector.yarn + +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) + extends ScanBuilder + with SupportsPushDownFilters with Serializable { + + override def build(): Scan = { + YarnAppScan(options, schema, pushed) + } + + private var pushed: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supportedFilter, unsupportedFilter) = filters.partition { + case _: org.apache.spark.sql.sources.EqualTo => true + case _: org.apache.spark.sql.sources.In => true + case _ => false + } + pushed = supportedFilter + unsupportedFilter + } + + override def pushedFilters(): Array[Filter] = pushed +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala index 01e68eb6a87..52a22d150bf 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala @@ -40,13 +40,15 @@ class YarnApplicationTable extends Table with SupportsRead { StructField("submit_time", LongType, nullable = false), StructField("launch_time", LongType, nullable = false), StructField("start_time", LongType, nullable = false), - StructField("finish_time", LongType, nullable = false))) + StructField("finish_time", LongType, nullable = false), + StructField("tracking_url", StringType, nullable = false), + StructField("original_tracking_url", StringType, nullable = false))) override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = - new YarnAppScan( + YarnAppScanBuilder( caseInsensitiveStringMap, schema()) } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index 47ae7adbc33..ee9a91769de 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -17,39 +17,11 @@ package org.apache.kyuubi.spark.connector.yarn -import java.io.{File, FileOutputStream} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} - trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { - def writeConfigToFile(conf: Configuration, filePath: String): Unit = { - val file = new File(filePath) - info(s"xml path: ${file.getAbsolutePath}") - val outputStream = new FileOutputStream(file) - try { - conf.writeXml(outputStream) - } finally { - outputStream.close() - } - } + protected val CONF_DIR: String = "tmp/hadoop-conf" override def beforeAll(): Unit = { super.beforeAll() - // get all conf and set up hadoop conf dir - if (!new File("tmp/hadoop-conf").exists()) new File("tmp/hadoop-conf").mkdirs() - writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/core-site.xml") - writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hadoop-conf/hdfs-site.xml") - if (!new File("tmp/hdfs-conf").exists()) new File("tmp/hdfs-conf").mkdirs() - writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hdfs-conf/core-site.xml") - writeConfigToFile(miniHdfsService.getHadoopConf, "tmp/hdfs-conf/hdfs-site.xml") - // init log dir and set permission - val fs = FileSystem.get(hdfsConf) - val logDir = new Path("/tmp/logs") - fs.mkdirs(logDir) - fs.setPermission(logDir, new org.apache.hadoop.fs.permission.FsPermission("777")) - info(s"hdfs web address: http://${fs.getConf.get("dfs.http.address")}") - fs.close() // mock app submit submitMockTasksInParallelTreeTimes() // log all conf @@ -60,7 +32,5 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { override def afterAll(): Unit = { super.afterAll() - // delete hadoop conf dir - } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala similarity index 79% rename from extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala index 44554ee3e36..37102a9cb5c 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala @@ -17,14 +17,17 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.yarn.client.api.YarnClient -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -class YarnQuerySuite extends SparkYarnConnectorWithYarn { - test("query for table apps") { +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession + +class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { + test("query apps") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") @@ -45,11 +48,35 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { val appCnt = spark.sql("select count(1) from yarn.default.apps") .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) assert(appCnt >= 0L) + } + } + + test("query app with appId") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") val yarnClient = YarnClient.createYarnClient() yarnClient.init(yarnConf) yarnClient.start() yarnClient.getApplications.forEach(app => { val appId = app.getApplicationId.toString + val queryCnt = spark.sql("select count(1) from yarn.default.apps " + + s"where id = '${appId}'").collect().head.getLong(0) + assert(queryCnt == 1) val queryApps = spark.sql(s"select * from yarn.default.apps " + s"where id = '${appId}'").collect() .map(x => x.asInstanceOf[GenericRowWithSchema]) @@ -65,8 +92,9 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { submitTime = x.getLong(7), launchTime = x.getLong(8), startTime = x.getLong(9), - finishTime = x.getLong(10))) - assert(!queryApps.isEmpty) + finishTime = x.getLong(10), + trackingUrl = x.getString(11), + originalTrackingUrl = x.getString(12))) val queryApp = queryApps.head assert(queryApp.appType == app.getApplicationType) assert(queryApp.user == app.getUser) @@ -78,12 +106,14 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { assert(queryApp.launchTime == app.getLaunchTime) assert(queryApp.startTime == app.getStartTime) assert(queryApp.finishTime == app.getFinishTime) + assert(queryApp.trackingUrl == app.getTrackingUrl) + assert(queryApp.originalTrackingUrl == app.getOriginalTrackingUrl) }) yarnClient.close() } } - test("use yarn conf with HADOOP_CONF") { + test("query app with equalTo appType") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") @@ -101,33 +131,22 @@ class YarnQuerySuite extends SparkYarnConnectorWithYarn { }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") - val appCnt = spark.sql("select count(1) from yarn.default.apps") - .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) - assert(appCnt >= 0L) + val yarnClient = YarnClient.createYarnClient() + yarnClient.init(yarnConf) + yarnClient.start() + yarnClient.getApplications().asScala.map(x => x.getYarnApplicationState.toString) + .map(x => (x, 1)) + .groupBy(x => x._1) + .mapValues(values => values.map(_._2).sum) + .foreach(x => { + val appState = x._1 + val appCnt = x._2 + val queryCnt = spark.sql("select count(1) from yarn.default.apps " + + s"where state = '${appState}'").collect().head.getLong(0) + assert(queryCnt == appCnt) + }) + yarnClient.close() } } - test("use yarn conf in hdfs") { - val sparkConf = new SparkConf() - .setMaster("local[*]") - .set("spark.ui.enabled", "false") - .set("spark.sql.catalogImplementation", "in-memory") - .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) - miniHdfsService.getHadoopConf.forEach(kv => { - if (kv.getKey.startsWith("fs")) { - sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) - } - }) - miniYarnService.getYarnConf.forEach(kv => { - if (kv.getKey.startsWith("yarn")) { - sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) - } - }) - withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - spark.sql("USE yarn") - val appCnt = spark.sql("select count(1) from yarn.default.apps") - .collect().head.asInstanceOf[GenericRowWithSchema].getLong(0) - assert(appCnt >= 0L) - } - } } From ce7930a42d1b90b41c7c52dd65c6a7eb7bf81df8 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Tue, 17 Dec 2024 22:47:53 +0800 Subject: [PATCH 13/38] predicates push down in app tables with equalTo(appType) --- .../spark/connector/yarn/YarnAppScan.scala | 4 -- .../connector/yarn/YarnAppQuerySuite.scala | 38 ++++++++++++++++++- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 6a68138f591..9c6882ad638 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -34,10 +34,6 @@ case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType, pu override def readSchema(): StructType = schema override def planInputPartitions(): Array[InputPartition] = { - // show pushed - // scalastyle:off println - println(s"Applying filters: ${pushed.mkString(", ")}") - // scalastyle:on println Array(YarnAppPartition( SparkSession.active.sparkContext .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap, diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala index 37102a9cb5c..45e0769bd23 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala @@ -113,7 +113,7 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { } } - test("query app with equalTo appType") { + test("query app with equalTo appState") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") @@ -149,4 +149,40 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { } } + test("query app with equalTo appType") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + val yarnClient = YarnClient.createYarnClient() + yarnClient.init(yarnConf) + yarnClient.start() + yarnClient.getApplications().asScala.map(x => x.getApplicationType) + .map(x => (x, 1)) + .groupBy(x => x._1) + .mapValues(values => values.map(_._2).sum) + .foreach(x => { + val appType = x._1 + val appCnt = x._2 + val queryCnt = spark.sql("select count(1) from yarn.default.apps " + + s"where type = '${appType}'").collect().head.getLong(0) + assert(queryCnt == appCnt) + }) + yarnClient.close() + } + } + } From c0dd20a6247a456155e51ea4b0d8def5b29a4d48 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Wed, 18 Dec 2024 21:08:33 +0800 Subject: [PATCH 14/38] predicates push down in app tables with in appType or in appState --- .../connector/yarn/YarnAppQuerySuite.scala | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala index 45e0769bd23..0d6178d23c6 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala @@ -17,14 +17,13 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.jdk.CollectionConverters.asScalaBufferConverter - import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession +import scala.jdk.CollectionConverters.asScalaBufferConverter class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { test("query apps") { @@ -113,7 +112,7 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { } } - test("query app with equalTo appState") { + test("query app with equalTo/in appState") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") @@ -141,15 +140,18 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { .foreach(x => { val appState = x._1 val appCnt = x._2 - val queryCnt = spark.sql("select count(1) from yarn.default.apps " + + val queryCntWithEqualTo = spark.sql("select count(1) from yarn.default.apps " + s"where state = '${appState}'").collect().head.getLong(0) - assert(queryCnt == appCnt) + assert(queryCntWithEqualTo == appCnt) + val queryCntWithIn = spark.sql("select count(1) from yarn.default.apps " + + s"where state in ('${appState}')").collect().head.getLong(0) + assert(queryCntWithIn == appCnt) }) yarnClient.close() } } - test("query app with equalTo appType") { + test("query app with equalTo/in appType") { val sparkConf = new SparkConf() .setMaster("local[*]") .set("spark.ui.enabled", "false") @@ -177,9 +179,12 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { .foreach(x => { val appType = x._1 val appCnt = x._2 - val queryCnt = spark.sql("select count(1) from yarn.default.apps " + + val queryCntWithEqualTo = spark.sql("select count(1) from yarn.default.apps " + s"where type = '${appType}'").collect().head.getLong(0) - assert(queryCnt == appCnt) + assert(queryCntWithEqualTo == appCnt) + val queryCntWithIn = spark.sql("select count(1) from yarn.default.apps " + + s"where type in ('${appType}')").collect().head.getLong(0) + assert(queryCntWithIn == appCnt) }) yarnClient.close() } From ebb7a2897726abc1cebe78620f6521e5f4b9f3f9 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Wed, 18 Dec 2024 23:55:23 +0800 Subject: [PATCH 15/38] refactor all log related scala codes --- .../spark/connector/yarn/BasicScan.scala | 35 ++++ .../connector/yarn/BasicScanBuilder.scala | 40 ++++ .../yarn/YarnAppPartitionReader.scala | 13 +- .../spark/connector/yarn/YarnAppScan.scala | 15 +- .../connector/yarn/YarnAppScanBuilder.scala | 18 +- .../spark/connector/yarn/YarnCatalog.scala | 2 +- ...Partition.scala => YarnLogPartition.scala} | 5 +- .../yarn/YarnLogPartitionReader.scala | 171 ++++++++++++++++++ ...ctory.scala => YarnLogReaderFactory.scala} | 6 +- .../{YarnLogsScan.scala => YarnLogScan.scala} | 13 +- .../connector/yarn/YarnLogScanBuilder.scala | 40 ++++ ...YarnLogsTable.scala => YarnLogTable.scala} | 17 +- .../yarn/YarnLogsPartitionReader.scala | 47 ----- .../yarn/SparkYarnConnectorWithYarn.scala | 5 +- .../WithKyuubiServerAndYarnMiniCluster.scala | 12 +- .../connector/yarn/YarnLogQuerySuite.scala | 49 +++++ 16 files changed, 376 insertions(+), 112 deletions(-) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScan.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/{YarnLogsPartition.scala => YarnLogPartition.scala} (81%) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/{YarnLogsReaderFactory.scala => YarnLogReaderFactory.scala} (85%) rename extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/{YarnLogsScan.scala => YarnLogScan.scala} (79%) create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala rename extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/{YarnLogsTable.scala => YarnLogTable.scala} (87%) delete mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala create mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScan.scala new file mode 100644 index 00000000000..75407211de3 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScan.scala @@ -0,0 +1,35 @@ +/* + * 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.spark.connector.yarn + +import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read._ + +trait BasicScan + extends ScanBuilder + with Scan with Batch with Serializable { + + protected val hadoopConfMap: Map[String, String] = SparkSession.active.sparkContext + .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap + + override def toBatch: Batch = this + + override def build(): Scan = this +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala new file mode 100644 index 00000000000..862716f81b9 --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala @@ -0,0 +1,40 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.Filter + +trait BasicScanBuilder + extends ScanBuilder + with SupportsPushDownFilters with Serializable { + + protected var pushed: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supportedFilter, unsupportedFilter) = filters.partition { + case _: org.apache.spark.sql.sources.EqualTo => true + case _: org.apache.spark.sql.sources.In => true + case _ => false + } + pushed = supportedFilter + unsupportedFilter + } + + override def pushedFilters(): Array[Filter] = pushed +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 4db1b4f8e2a..8a27dbf783b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,10 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient @@ -31,6 +27,10 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.sources.{EqualTo, In} import org.apache.spark.unsafe.types.UTF8String +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { @@ -88,11 +88,6 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) }.get } - // case yarnAppPartition.filters match { - // case filters if filters.isEmpty => yarnClient.getApplications - // case filters => - // yarnClient.getApplications - // } val appSeq = applicationReports.asScala.map(app => { YarnApplication( id = app.getApplicationId.toString, diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala index 9c6882ad638..faa83e3cc39 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.scala @@ -17,33 +17,26 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.jdk.CollectionConverters.iterableAsScalaIterableConverter - -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap case class YarnAppScan(options: CaseInsensitiveStringMap, schema: StructType, pushed: Array[Filter]) - extends ScanBuilder - with Scan with Batch with Serializable { + extends BasicScan { - override def toBatch: Batch = this + // override def toBatch: Batch = this override def readSchema(): StructType = schema override def planInputPartitions(): Array[InputPartition] = { Array(YarnAppPartition( - SparkSession.active.sparkContext - .hadoopConfiguration.asScala.map(kv => (kv.getKey, kv.getValue)).toMap, + hadoopConfMap, pushed)) } override def createReaderFactory(): PartitionReaderFactory = new YarnAppReaderFactory - override def build(): Scan = { - this - } + // override def build(): Scan = this } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala index b0fc0c391a2..3af6112354b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala @@ -18,29 +18,13 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) - extends ScanBuilder - with SupportsPushDownFilters with Serializable { + extends BasicScanBuilder { override def build(): Scan = { YarnAppScan(options, schema, pushed) } - - private var pushed: Array[Filter] = Array.empty - - override def pushFilters(filters: Array[Filter]): Array[Filter] = { - val (supportedFilter, unsupportedFilter) = filters.partition { - case _: org.apache.spark.sql.sources.EqualTo => true - case _: org.apache.spark.sql.sources.In => true - case _ => false - } - pushed = supportedFilter - unsupportedFilter - } - - override def pushedFilters(): Array[Filter] = pushed } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 24a0969554e..1471d2f2ea6 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -46,7 +46,7 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { } override def loadTable(identifier: Identifier): Table = identifier.name match { - case "app_logs" => new YarnLogsTable + case "app_logs" => new YarnLogTable case "apps" => new YarnApplicationTable case _ => throw new NoSuchTableException(s"${identifier.name}") diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala similarity index 81% rename from extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala index 5cc61b6cb4d..511d2e8089a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala @@ -18,5 +18,8 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.sources.Filter -class YarnLogsPartition(val appId: String) extends InputPartition +case class YarnLogPartition(hadoopConfMap: Map[String, String], filters: Array[Filter], + logPath: Option[String]) + extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala new file mode 100644 index 00000000000..8f3c39a315a --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -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.kyuubi.spark.connector.yarn + +import java.io.{BufferedReader, InputStreamReader} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.io.IOUtils +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.sql.sources.EqualTo + +class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) + extends PartitionReader[InternalRow] { + + private val remoteAppLogDirKey = "yarn.nodemanager.remote-app-log-dir" + + private val fs = { + val hadoopConf = new Configuration() + yarnLogPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) + FileSystem.get(hadoopConf) + } + + private val logsIterator = fetchLogs().iterator + + override def next(): Boolean = logsIterator.hasNext + + override def get(): InternalRow = { + val log = logsIterator.next() + // new GenericInternalRow(Array[Any](log.appId, log.logLevel, log.message)) + null + } + + // given a path in hdfs, then get all files under it, supports * + def listFiles(pathStr: String): mutable.Seq[FileStatus] = { + val path = new Path(pathStr) + val logFiles = mutable.ArrayBuffer[FileStatus]() + if (fs.exists(path)) { + val fileStatuses: Array[FileStatus] = fs.globStatus(path) + if (fileStatuses != null && fileStatuses.nonEmpty) { + fileStatuses.foreach { + case status if status.isFile => logFiles += status + case status if status.isDirectory => + val fileIterator = fs.listFiles(status.getPath, true) + while (fileIterator.hasNext) { + val fileStatus = fileIterator.next() + if (fileStatus.isFile) logFiles += fileStatus + } + } + } + } + logFiles + } + + // /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 + // /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 + // /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 + + override def close(): Unit = { + fs.close() + } + + private def fetchLogs(): Seq[LogEntry] = { + // Simulate fetching logs for the given appId (replace with Yarn API calls) + val remoteAppLogDir = yarnLogPartition.logPath match { + case Some(dir) => Some(dir) + case _ => yarnLogPartition.hadoopConfMap.get(remoteAppLogDirKey) + } + // TODO throw exception here + val logFileStatuses = remoteAppLogDir match { + case Some(dir) => + yarnLogPartition.filters match { + case filters if filters.isEmpty => listFiles(remoteAppLogDir.get) + case filters => filters.collectFirst { + case EqualTo("app_id", appId: String) => + listFiles(s"${remoteAppLogDir}/*/*/*/*/${appId}") + // TODO hadoop2 listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") + case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") + case _ => listFiles(remoteAppLogDir.get) + }.get + } + case _ => mutable.Seq.empty + } + val logEntries = new ArrayBuffer[LogEntry]() + logFileStatuses.foreach { logFileStatus => + { + val split = logFileStatus.getPath.toString.split("/") + val containerId = split(split.length - 1) + val applicationId = split(split.length - 2) + // TODO use regexp + val user = remoteAppLogDir.get match { + case dir if dir.startsWith("hdfs") => + logFileStatus.getPath.toString.split(s"${dir}")(0).split("/")(0) + case dir => logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(0) + } + logFileStatus.getPath.toString.split(s"${remoteAppLogDir}")(1).split("/")(0) + // todo read logs multi-threads + logEntries ++= fetchLog( + logFileStatus, + user, + containerId, + applicationId) + } + } + Seq() + } + + /** + * fet log + * @param logStatus + * @param user + * @param containerId + * @param applicationId + */ + private def fetchLog( + logStatus: FileStatus, + user: String, + containerId: String, + applicationId: String): Seq[LogEntry] = { + val path = logStatus.getPath + val inputStream = fs.open(path) + val reader = new BufferedReader(new InputStreamReader(inputStream)) + var line: String = null + var lineNumber: Int = 1 + val logEntries = new ArrayBuffer[LogEntry]() + try { + while ({ line = reader.readLine(); line != null }) { + // println(s"Line $lineNumber: $line") + lineNumber += 1 + logEntries += LogEntry( + applicationId, + user, + containerId, + lineNumber, + line) + } + logEntries + } finally { + // 关闭流 + IOUtils.closeStream(inputStream) + reader.close() + } + } +} + +// Helper class to represent log entries +case class LogEntry( + appId: String, + user: String, + containerId: String, + rowNumber: Int, + message: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogReaderFactory.scala similarity index 85% rename from extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogReaderFactory.scala index 1d7d97d0ed6..0b21a381fa5 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsReaderFactory.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogReaderFactory.scala @@ -20,10 +20,10 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} -class YarnLogsReaderFactory extends PartitionReaderFactory { +class YarnLogReaderFactory extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val yarnPartition = partition.asInstanceOf[YarnLogsPartition] - new YarnLogsPartitionReader(yarnPartition.appId) + val yarnPartition = partition.asInstanceOf[YarnLogPartition] + new YarnLogPartitionReader(yarnPartition) } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala similarity index 79% rename from extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index fd680189d82..44f6353f515 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -18,21 +18,22 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnLogsScan(options: CaseInsensitiveStringMap, schema: StructType) extends ScanBuilder - with Scan with Batch with Serializable { - private val appId: String = options.getOrDefault("appId", "*") +case class YarnLogScan(options: CaseInsensitiveStringMap, schema: StructType, + filters: Array[Filter]) + extends BasicScan { override def readSchema(): StructType = schema override def planInputPartitions(): Array[InputPartition] = { // Fetch logs for the given appId (filtering logic can be added) - Array(new YarnLogsPartition(appId)) + // TODO make remote dir configurable + Array(YarnLogPartition(hadoopConfMap, filters, null)) } override def createReaderFactory(): PartitionReaderFactory = - new YarnLogsReaderFactory + new YarnLogReaderFactory - override def build(): Scan = this } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala new file mode 100644 index 00000000000..57c6679414c --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala @@ -0,0 +1,40 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) + extends BasicScanBuilder { + + override def build(): Scan = { + YarnLogScan(options, schema, pushed) + } + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supportedFilter, unsupportedFilter) = filters.partition { + case _: org.apache.spark.sql.sources.EqualTo => true + case _ => false + } + pushed = supportedFilter + unsupportedFilter + } +} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala similarity index 87% rename from extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala rename to extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index 3d30793237e..0edffbe6c47 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -16,30 +16,29 @@ */ package org.apache.kyuubi.spark.connector.yarn -import java.util - -import scala.jdk.CollectionConverters.setAsJavaSetConverter - import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnLogsTable extends Table with SupportsRead { - override def name(): String = "app_log" +import java.util +import scala.jdk.CollectionConverters.setAsJavaSetConverter + +class YarnLogTable extends Table with SupportsRead { + override def name(): String = "app_logs" override def schema(): StructType = new StructType(Array( - StructField("appId", StringType, nullable = false), + StructField("app_id", StringType, nullable = false), StructField("user", StringType, nullable = false), - StructField("rowIndex", IntegerType, nullable = false), + StructField("row_number", IntegerType, nullable = false), StructField("message", StringType, nullable = true))) override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = - new YarnLogsScan( + YarnLogScanBuilder( caseInsensitiveStringMap, schema()) } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala deleted file mode 100644 index 6e4b6899fd8..00000000000 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogsPartitionReader.scala +++ /dev/null @@ -1,47 +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.kyuubi.spark.connector.yarn - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.connector.read.PartitionReader - -class YarnLogsPartitionReader(appId: String) extends PartitionReader[InternalRow] { - - private val logsIterator = fetchLogs(appId).iterator - - override def next(): Boolean = logsIterator.hasNext - - override def get(): InternalRow = { - val log = logsIterator.next() - new GenericInternalRow(Array[Any](log.appId, log.logLevel, log.message)) - } - - override def close(): Unit = {} - - private def fetchLogs(appId: String): Seq[LogEntry] = { - // Simulate fetching logs for the given appId (replace with Yarn API calls) - Seq( - LogEntry(appId, "INFO", "Application started"), - LogEntry(appId, "WARN", "Low memory"), - LogEntry(appId, "ERROR", "Application failed")) - } -} - -// Helper class to represent log entries -case class LogEntry(appId: String, logLevel: String, message: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index ee9a91769de..fb660603657 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -25,9 +25,10 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { // mock app submit submitMockTasksInParallelTreeTimes() // log all conf - miniHdfsService.getHadoopConf.forEach(kv => + miniHdfsService.getHadoopConf.forEach( + kv => info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) + miniYarnService.getYarnConf.forEach(kv => info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) - miniYarnService.getYarnConf.forEach(kv => info(s"mini yarn conf ${kv.getKey}: ${kv.getValue}")) } override def afterAll(): Unit = { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index a240dbdf64b..248788f168a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -54,8 +54,8 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096) yarnConfig.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true) - yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1) - yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 10) + yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 5) + yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 20) yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4) yarnConfig.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600) yarnConfig.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false) @@ -90,7 +90,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS // Configure YARN log aggregation yarnConfig.set("yarn.nodemanager.remote-app-log-dir", "/tmp/logs") - yarnConfig.set("yarn.nodemanager.remote-app-log-dir-suffix", "logs") + yarnConfig.set("yarn.nodemanager.remote-app-log-dir-suffix", "xxx") yarnConfig.set("yarn.log-aggregation-enable", "true") yarnConfig.set("yarn.log-aggregation.retain-seconds", "3600") yarnConfig.set("yarn.log-aggregation.retain-check-interval-seconds", "300") @@ -106,7 +106,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS hdfsConf.set("dfs.datanode.metrics.logger.period.seconds", "0") // TODO delete it - hdfsConf.set("hadoop.http.staticuser.user", s"zhangxinsen") + hdfsConf.set("hadoop.http.staticuser.user", "zhangxinsen") miniHdfsService = new MiniDFSService(hdfsConf) miniHdfsService.initialize(conf) @@ -160,7 +160,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS // Set up container launch context (e.g., commands to execute) val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - val commands = Collections.singletonList("echo Hello, MiniYARNCluster! && sleep 5") + val commands = Collections.singletonList("echo Hello, MiniYARNCluster! && sleep 5 lscd") amContainer.setCommands(commands) // Application Master resource requirements @@ -192,7 +192,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS } def submitMockTasksInParallelTreeTimes(): Unit = { - val threads = (1 to 3).map { i => + val threads = (1 to 10).map { i => new Thread(() => { info(s"Starting submission in thread $i") try { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala new file mode 100644 index 00000000000..913ca4ba35a --- /dev/null +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -0,0 +1,49 @@ +/* + * 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.spark.connector.yarn + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession + +class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { + test("query logs") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + val rows = spark.sql("select * from yarn.default.app_logs").collect() + rows + } + } + +} From d1192d8a58f11c2f4872e1d89862977e50a336d5 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Wed, 18 Dec 2024 23:55:59 +0800 Subject: [PATCH 16/38] refactor all log related scala codes --- .../spark/connector/yarn/YarnAppPartitionReader.scala | 8 ++++---- .../kyuubi/spark/connector/yarn/YarnLogPartition.scala | 6 ++++-- .../apache/kyuubi/spark/connector/yarn/YarnLogScan.scala | 6 ++++-- .../apache/kyuubi/spark/connector/yarn/YarnLogTable.scala | 7 ++++--- .../spark/connector/yarn/SparkYarnConnectorWithYarn.scala | 4 ++-- .../kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala | 5 +++-- 6 files changed, 21 insertions(+), 15 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 8a27dbf783b..6aaaadc01ae 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,6 +17,10 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient @@ -27,10 +31,6 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.sources.{EqualTo, In} import org.apache.spark.unsafe.types.UTF8String -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala index 511d2e8089a..bb78ad693f7 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala @@ -20,6 +20,8 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.sources.Filter -case class YarnLogPartition(hadoopConfMap: Map[String, String], filters: Array[Filter], - logPath: Option[String]) +case class YarnLogPartition( + hadoopConfMap: Map[String, String], + filters: Array[Filter], + logPath: Option[String]) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index 44f6353f515..3bae357a50f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -22,8 +22,10 @@ import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -case class YarnLogScan(options: CaseInsensitiveStringMap, schema: StructType, - filters: Array[Filter]) +case class YarnLogScan( + options: CaseInsensitiveStringMap, + schema: StructType, + filters: Array[Filter]) extends BasicScan { override def readSchema(): StructType = schema diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index 0edffbe6c47..c6772393bba 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -16,14 +16,15 @@ */ package org.apache.kyuubi.spark.connector.yarn +import java.util + +import scala.jdk.CollectionConverters.setAsJavaSetConverter + import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -import java.util -import scala.jdk.CollectionConverters.setAsJavaSetConverter - class YarnLogTable extends Table with SupportsRead { override def name(): String = "app_logs" diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala index fb660603657..f32a2ab463c 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/SparkYarnConnectorWithYarn.scala @@ -25,8 +25,8 @@ trait SparkYarnConnectorWithYarn extends WithKyuubiServerAndYarnMiniCluster { // mock app submit submitMockTasksInParallelTreeTimes() // log all conf - miniHdfsService.getHadoopConf.forEach( - kv => info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) + miniHdfsService.getHadoopConf.forEach(kv => + info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) miniYarnService.getYarnConf.forEach(kv => info(s"mini hdfs conf ${kv.getKey}: ${kv.getValue}")) } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala index 0d6178d23c6..1b79e3d15ee 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala @@ -17,13 +17,14 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.yarn.client.api.YarnClient -import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import scala.jdk.CollectionConverters.asScalaBufferConverter +import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { test("query apps") { From 7ef1fd377b4f3eec1831a3958af46539ce7e8ebe Mon Sep 17 00:00:00 2001 From: native-zhang Date: Thu, 19 Dec 2024 00:08:10 +0800 Subject: [PATCH 17/38] refactor all log related scala codes --- .../yarn/YarnLogPartitionReader.scala | 25 +++++++++++-------- .../spark/connector/yarn/YarnLogTable.scala | 1 + .../connector/yarn/YarnLogQuerySuite.scala | 2 +- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 8f3c39a315a..eb59db9eb51 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -26,8 +26,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.io.IOUtils import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.sources.EqualTo +import org.apache.spark.unsafe.types.UTF8String class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { @@ -40,14 +42,18 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) FileSystem.get(hadoopConf) } - private val logsIterator = fetchLogs().iterator + private val logIterator = fetchLogs().iterator - override def next(): Boolean = logsIterator.hasNext + override def next(): Boolean = logIterator.hasNext override def get(): InternalRow = { - val log = logsIterator.next() - // new GenericInternalRow(Array[Any](log.appId, log.logLevel, log.message)) - null + val yarnLog = logIterator.next() + new GenericInternalRow(Array[Any]( + UTF8String.fromString(yarnLog.appId), + UTF8String.fromString(yarnLog.user), + UTF8String.fromString(yarnLog.containerId), + yarnLog.rowNumber, + UTF8String.fromString(yarnLog.message))) } // given a path in hdfs, then get all files under it, supports * @@ -108,11 +114,10 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) val applicationId = split(split.length - 2) // TODO use regexp val user = remoteAppLogDir.get match { - case dir if dir.startsWith("hdfs") => - logFileStatus.getPath.toString.split(s"${dir}")(0).split("/")(0) - case dir => logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(0) + case dir if dir.endsWith("/") => + logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(0) + case dir => logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(1) } - logFileStatus.getPath.toString.split(s"${remoteAppLogDir}")(1).split("/")(0) // todo read logs multi-threads logEntries ++= fetchLog( logFileStatus, @@ -121,7 +126,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) applicationId) } } - Seq() + logEntries } /** diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index c6772393bba..83dba61ffaf 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -32,6 +32,7 @@ class YarnLogTable extends Table with SupportsRead { new StructType(Array( StructField("app_id", StringType, nullable = false), StructField("user", StringType, nullable = false), + StructField("container_id", StringType, nullable = false), StructField("row_number", IntegerType, nullable = false), StructField("message", StringType, nullable = true))) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index 913ca4ba35a..b0fd77b01c3 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -42,7 +42,7 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") val rows = spark.sql("select * from yarn.default.app_logs").collect() - rows + rows.foreach(row => info(row.toString())) } } From 0faade607da5ddfd6b288eb255f7817ead34dde5 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Fri, 20 Dec 2024 02:48:18 +0800 Subject: [PATCH 18/38] add todo for modify task nums --- .../kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala | 2 -- .../org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala | 1 + 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index eb59db9eb51..2403e8156b6 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -118,7 +118,6 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(0) case dir => logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(1) } - // todo read logs multi-threads logEntries ++= fetchLog( logFileStatus, user, @@ -160,7 +159,6 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) } logEntries } finally { - // 关闭流 IOUtils.closeStream(inputStream) reader.close() } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index 3bae357a50f..5467cb73405 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -32,6 +32,7 @@ case class YarnLogScan( override def planInputPartitions(): Array[InputPartition] = { // Fetch logs for the given appId (filtering logic can be added) // TODO make remote dir configurable + // TODO get file nums and construct nums inputPartition Array(YarnLogPartition(hadoopConfMap, filters, null)) } From 445c4052f6fc686168eb67b5c978402a7c814009 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 00:40:34 +0800 Subject: [PATCH 19/38] try to push down predicates for log reading --- .../connector/yarn/YarnLogPartition.scala | 5 +- .../yarn/YarnLogPartitionReader.scala | 150 ++++++------------ .../spark/connector/yarn/YarnLogScan.scala | 85 +++++++++- .../spark/connector/yarn/YarnLogTable.scala | 1 + .../connector/yarn/YarnLogQuerySuite.scala | 10 +- 5 files changed, 139 insertions(+), 112 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala index bb78ad693f7..7c9a0683693 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala @@ -18,10 +18,9 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition -import org.apache.spark.sql.sources.Filter case class YarnLogPartition( hadoopConfMap: Map[String, String], - filters: Array[Filter], - logPath: Option[String]) + logPath: String, + remoteAppLogDir: String) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 2403e8156b6..93d1c167b83 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -19,30 +19,27 @@ package org.apache.kyuubi.spark.connector.yarn import java.io.{BufferedReader, InputStreamReader} -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.IOUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader -import org.apache.spark.sql.sources.EqualTo import org.apache.spark.unsafe.types.UTF8String class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { - private val remoteAppLogDirKey = "yarn.nodemanager.remote-app-log-dir" - private val fs = { val hadoopConf = new Configuration() yarnLogPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) FileSystem.get(hadoopConf) } - private val logIterator = fetchLogs().iterator + private val logIterator = fetchLog().iterator override def next(): Boolean = logIterator.hasNext @@ -51,117 +48,69 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) new GenericInternalRow(Array[Any]( UTF8String.fromString(yarnLog.appId), UTF8String.fromString(yarnLog.user), + UTF8String.fromString(yarnLog.host), UTF8String.fromString(yarnLog.containerId), yarnLog.rowNumber, UTF8String.fromString(yarnLog.message))) } - // given a path in hdfs, then get all files under it, supports * - def listFiles(pathStr: String): mutable.Seq[FileStatus] = { - val path = new Path(pathStr) - val logFiles = mutable.ArrayBuffer[FileStatus]() - if (fs.exists(path)) { - val fileStatuses: Array[FileStatus] = fs.globStatus(path) - if (fileStatuses != null && fileStatuses.nonEmpty) { - fileStatuses.foreach { - case status if status.isFile => logFiles += status - case status if status.isDirectory => - val fileIterator = fs.listFiles(status.getPath, true) - while (fileIterator.hasNext) { - val fileStatus = fileIterator.next() - if (fileStatus.isFile) logFiles += fileStatus - } - } - } - } - logFiles - } - - // /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 - // /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 - // /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 - override def close(): Unit = { fs.close() } - private def fetchLogs(): Seq[LogEntry] = { - // Simulate fetching logs for the given appId (replace with Yarn API calls) - val remoteAppLogDir = yarnLogPartition.logPath match { - case Some(dir) => Some(dir) - case _ => yarnLogPartition.hadoopConfMap.get(remoteAppLogDirKey) - } - // TODO throw exception here - val logFileStatuses = remoteAppLogDir match { - case Some(dir) => - yarnLogPartition.filters match { - case filters if filters.isEmpty => listFiles(remoteAppLogDir.get) - case filters => filters.collectFirst { - case EqualTo("app_id", appId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/*/${appId}") - // TODO hadoop2 listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") - case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") - case _ => listFiles(remoteAppLogDir.get) - }.get - } - case _ => mutable.Seq.empty - } - val logEntries = new ArrayBuffer[LogEntry]() - logFileStatuses.foreach { logFileStatus => - { - val split = logFileStatus.getPath.toString.split("/") - val containerId = split(split.length - 1) - val applicationId = split(split.length - 2) - // TODO use regexp - val user = remoteAppLogDir.get match { - case dir if dir.endsWith("/") => - logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(0) - case dir => logFileStatus.getPath.toString.split(s"${dir}")(1).split("/")(1) - } - logEntries ++= fetchLog( - logFileStatus, - user, - containerId, - applicationId) - } - } - logEntries - } - /** * fet log + * * hadoop3: + * * /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 + * * /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 + * * hadoop2: + * * /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 + * * @param logStatus * @param user * @param containerId * @param applicationId */ - private def fetchLog( - logStatus: FileStatus, - user: String, - containerId: String, - applicationId: String): Seq[LogEntry] = { - val path = logStatus.getPath - val inputStream = fs.open(path) - val reader = new BufferedReader(new InputStreamReader(inputStream)) - var line: String = null - var lineNumber: Int = 1 - val logEntries = new ArrayBuffer[LogEntry]() - try { - while ({ line = reader.readLine(); line != null }) { - // println(s"Line $lineNumber: $line") - lineNumber += 1 - logEntries += LogEntry( - applicationId, - user, - containerId, - lineNumber, - line) - } - logEntries - } finally { - IOUtils.closeStream(inputStream) - reader.close() + private def fetchLog(): Seq[LogEntry] = { + val logDirInReg = yarnLogPartition.remoteAppLogDir match { + // in case of /tmp/logs/, /tmp/logs// + case dir if dir.endsWith("/") => dir.replaceAll("/+", "/").replace("/", "\\/") + // in case of /tmp/logs + case dir => dir.replace("/", "\\/") + } + val pathPattern: Regex = + s".*${logDirInReg}/(.*?)/.*?/(application_.*?)/(.+)_(\\d+)".r + yarnLogPartition.logPath match { + case pathPattern(user, applicationId, containerHost, containerSuffix) => + val path = new Path(yarnLogPartition.logPath) + val inputStream = fs.open(path) + val reader = new BufferedReader(new InputStreamReader(inputStream)) + var line: String = null + var lineNumber: Int = 1 + val logEntries = new ArrayBuffer[LogEntry]() + try { + while ({ + line = reader.readLine(); + line != null + }) { + // println(s"Line $lineNumber: $line") + lineNumber += 1 + logEntries += LogEntry( + applicationId, + user, + s"${containerHost}_${containerSuffix}", + containerHost, + lineNumber, + line) + } + logEntries + } finally { + IOUtils.closeStream(inputStream) + reader.close() + } + case _ => Seq.empty } + } } @@ -170,5 +119,6 @@ case class LogEntry( appId: String, user: String, containerId: String, + host: String, rowNumber: Int, message: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index 5467cb73405..abbb6291cfe 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -17,8 +17,13 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -29,11 +34,81 @@ case class YarnLogScan( extends BasicScan { override def readSchema(): StructType = schema + private val remoteAppLogDirKeyInYarnSite = "yarn.nodemanager.remote-app-log-dir" + private val remoteAppLogDirKey = "spark.sql.catalog.yarn.log.dir" + + private val remoteAppLogDir = { + val dir = SparkSession.active.sparkContext + .getConf.getOption(remoteAppLogDirKey) match { + case Some(dir) => Some(dir) + case _ => hadoopConfMap.get(remoteAppLogDirKeyInYarnSite) + } + if (dir.isEmpty) { + throw new IllegalArgumentException( + s"remoteAppLogDir should be set with ${remoteAppLogDirKey} or set with " + + s"${remoteAppLogDirKeyInYarnSite} in yarn-site.xml") + } + dir.get + } + + // given a path in hdfs, then get all files under it, supports * + private def listFiles(pathStr: String): mutable.Seq[FileStatus] = { + val hadoopConf = new Configuration() + hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) + val fs = FileSystem.get(hadoopConf) + val path = new Path(pathStr) + val logFiles = mutable.ArrayBuffer[FileStatus]() + if (fs.exists(path)) { + val fileStatuses: Array[FileStatus] = fs.globStatus(path) + if (fileStatuses != null && fileStatuses.nonEmpty) { + fileStatuses.foreach { + case status if status.isFile => logFiles += status + case status if status.isDirectory => + val fileIterator = fs.listFiles(status.getPath, true) + while (fileIterator.hasNext) { + val fileStatus = fileIterator.next() + if (fileStatus.isFile) logFiles += fileStatus + } + } + } + } + fs.close() + logFiles + } + + /** + * pushdown equalTo + * hadoop3: + * /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 + * /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 + * hadoop2: + * /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 + * + * @return + */ + private def tryPushDownPredicates(): mutable.Seq[FileStatus] = { + filters match { + case pushed if pushed.isEmpty => listFiles(remoteAppLogDir) + case pushed => pushed.collectFirst { + case EqualTo("app_id", appId: String) => + listFiles(s"${remoteAppLogDir}/*/*/*/*/${appId}") ++ + // compatible for hadoop2 + listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") + case EqualTo("container_id", containerId: String) => + listFiles(s"${remoteAppLogDir}/*/*/*/*/*/${containerId}") ++ + // compatible for hadoop2 + listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") + case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") + case _ => listFiles(remoteAppLogDir) + }.get + } + } + override def planInputPartitions(): Array[InputPartition] = { - // Fetch logs for the given appId (filtering logic can be added) - // TODO make remote dir configurable - // TODO get file nums and construct nums inputPartition - Array(YarnLogPartition(hadoopConfMap, filters, null)) + // get file nums and construct nums inputPartition + tryPushDownPredicates().map(fileStatus => { + YarnLogPartition(hadoopConfMap, fileStatus.getPath.toString, remoteAppLogDir) + }).toArray } override def createReaderFactory(): PartitionReaderFactory = diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index 83dba61ffaf..c23d59a3c8f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -32,6 +32,7 @@ class YarnLogTable extends Table with SupportsRead { new StructType(Array( StructField("app_id", StringType, nullable = false), StructField("user", StringType, nullable = false), + StructField("host", StringType, nullable = false), StructField("container_id", StringType, nullable = false), StructField("row_number", IntegerType, nullable = false), StructField("message", StringType, nullable = true))) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index b0fd77b01c3..069cd8fab8d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.SparkSession import org.apache.kyuubi.spark.connector.common.LocalSparkSession.withSparkSession class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { - test("query logs") { + test("query logs with host") { val sparkConf = new SparkConf() .setMaster("local[*]") - .set("spark.ui.enabled", "false") + // .set("spark.ui.enabled", "false") .set("spark.sql.catalogImplementation", "in-memory") .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) miniHdfsService.getHadoopConf.forEach(kv => { @@ -41,8 +41,10 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") - val rows = spark.sql("select * from yarn.default.app_logs").collect() - rows.foreach(row => info(row.toString())) + val cnt = spark.sql( + "select count(1) from yarn.default.app_logs where host='localhost'").collect().head.getLong( + 0) + assert(cnt > 0) } } From 94a570de4b4ef5c0b454d1a84cc343dac0ed548f Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 01:28:17 +0800 Subject: [PATCH 20/38] fix list tables error --- .../yarn/YarnAppPartitionReader.scala | 17 ++++++++++--- .../spark/connector/yarn/YarnCatalog.scala | 6 +---- .../yarn/YarnLogPartitionReader.scala | 1 - .../WithKyuubiServerAndYarnMiniCluster.scala | 5 +--- .../connector/yarn/YarnLogQuerySuite.scala | 24 +++++++++++++++++++ 5 files changed, 40 insertions(+), 13 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 6aaaadc01ae..be9d046bd8b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -34,6 +34,9 @@ import org.apache.spark.unsafe.types.UTF8String class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { + private val validYarnStateSet = + Set("NEW", "NEW_SAVING", "SUBMITTED", "ACCEPTED", "RUNNING", "FINISHED", "FAILED", "KILLED") + private val appIterator = fetchApp().iterator override def next(): Boolean = appIterator.hasNext @@ -76,12 +79,20 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) case EqualTo("id", appId: String) => java.util.Collections.singletonList( yarnClient.getApplicationReport(ApplicationId.fromString(appId))) case EqualTo("state", state: String) => - yarnClient.getApplications(java.util.EnumSet.of(YarnApplicationState.valueOf(state))) + state.toUpperCase match { + case validState if validYarnStateSet.contains(validState) => + yarnClient.getApplications( + java.util.EnumSet.of(YarnApplicationState.valueOf(validState))) + case _ => java.util.Collections.EMPTY_LIST[ApplicationReport] + } case EqualTo("type", appType: String) => yarnClient.getApplications(java.util.Collections.singleton(appType)) case In("state", states: Array[Any]) => yarnClient.getApplications( - java.util.EnumSet.copyOf(states.map(x => - YarnApplicationState.valueOf(x.toString)).toList.asJava)) + java.util.EnumSet.copyOf(states + .map(x => x.toString.toUpperCase) + .filter(x => validYarnStateSet.contains(x)) + .map(x => + YarnApplicationState.valueOf(x)).toList.asJava)) case In("type", types: Array[Any]) => yarnClient.getApplications( types.map(x => x.toString).toSet.asJava) case _ => yarnClient.getApplications() diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index 1471d2f2ea6..a6f6f742517 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -38,11 +38,7 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { } override def listTables(namespace: Array[String]): Array[Identifier] = { - namespace(1) match { - case "default" => - Array(Identifier.of(namespace, "app_logs"), Identifier.of(namespace, "apps")) - case _ => throw new NoSuchNamespaceException(namespace) - } + Array(Identifier.of(namespace, "app_logs"), Identifier.of(namespace, "apps")) } override def loadTable(identifier: Identifier): Table = identifier.name match { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 93d1c167b83..fcf40149655 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -110,7 +110,6 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) } case _ => Seq.empty } - } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index 248788f168a..60c291c1119 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -105,9 +105,6 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS hdfsConf.set("dfs.namenode.metrics.logger.period.seconds", "0") hdfsConf.set("dfs.datanode.metrics.logger.period.seconds", "0") - // TODO delete it - hdfsConf.set("hadoop.http.staticuser.user", "zhangxinsen") - miniHdfsService = new MiniDFSService(hdfsConf) miniHdfsService.initialize(conf) miniHdfsService.start() @@ -192,7 +189,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS } def submitMockTasksInParallelTreeTimes(): Unit = { - val threads = (1 to 10).map { i => + val threads = (1 to 100).map { i => new Thread(() => { info(s"Starting submission in thread $i") try { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index 069cd8fab8d..0599e18dbb4 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -48,4 +48,28 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { } } + test("query logs") { + val sparkConf = new SparkConf() + .setMaster("local[*]") + // .set("spark.ui.enabled", "false") + .set("spark.sql.catalogImplementation", "in-memory") + .set("spark.sql.catalog.yarn", classOf[YarnCatalog].getName) + miniHdfsService.getHadoopConf.forEach(kv => { + if (kv.getKey.startsWith("fs")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + miniYarnService.getYarnConf.forEach(kv => { + if (kv.getKey.startsWith("yarn")) { + sparkConf.set(s"spark.hadoop.${kv.getKey}", kv.getValue) + } + }) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + spark.sql("USE yarn") + val host = spark.sql( + "select * from yarn.default.app_logs limit 10").collect().head.getString(2) + assert(host == "localhost") + } + } + } From 2efb0dfa6d9e5388057e9c00e1c356b004b557ae Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 01:43:11 +0800 Subject: [PATCH 21/38] rename row_number into line_num and add file_name column in the table of app_logs --- .../connector/yarn/YarnAppPartitionReader.scala | 2 +- .../connector/yarn/YarnLogPartitionReader.scala | 16 +++++++++------- .../spark/connector/yarn/YarnLogTable.scala | 3 ++- .../WithKyuubiServerAndYarnMiniCluster.scala | 2 +- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index be9d046bd8b..cd757c1014e 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -83,7 +83,7 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) case validState if validYarnStateSet.contains(validState) => yarnClient.getApplications( java.util.EnumSet.of(YarnApplicationState.valueOf(validState))) - case _ => java.util.Collections.EMPTY_LIST[ApplicationReport] + case _ => Seq.empty[ApplicationReport].asJava } case EqualTo("type", appType: String) => yarnClient.getApplications(java.util.Collections.singleton(appType)) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index fcf40149655..32a0aed9366 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -17,11 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import java.io.{BufferedReader, InputStreamReader} - -import scala.collection.mutable.ArrayBuffer -import scala.util.matching.Regex - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.IOUtils @@ -30,6 +25,10 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String +import java.io.{BufferedReader, InputStreamReader} +import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex + class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { @@ -50,7 +49,8 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) UTF8String.fromString(yarnLog.user), UTF8String.fromString(yarnLog.host), UTF8String.fromString(yarnLog.containerId), - yarnLog.rowNumber, + yarnLog.lineNumber, + UTF8String.fromString(yarnLog.fileName), UTF8String.fromString(yarnLog.message))) } @@ -101,6 +101,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) s"${containerHost}_${containerSuffix}", containerHost, lineNumber, + path.getName, line) } logEntries @@ -119,5 +120,6 @@ case class LogEntry( user: String, containerId: String, host: String, - rowNumber: Int, + lineNumber: Int, + fileName: String, message: String) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index c23d59a3c8f..dc9fa3899c9 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -34,7 +34,8 @@ class YarnLogTable extends Table with SupportsRead { StructField("user", StringType, nullable = false), StructField("host", StringType, nullable = false), StructField("container_id", StringType, nullable = false), - StructField("row_number", IntegerType, nullable = false), + StructField("line_num", IntegerType, nullable = false), + StructField("file_name", StringType, nullable = false), StructField("message", StringType, nullable = true))) override def capabilities(): util.Set[TableCapability] = diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index 60c291c1119..78c11badfc9 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -189,7 +189,7 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS } def submitMockTasksInParallelTreeTimes(): Unit = { - val threads = (1 to 100).map { i => + val threads = (1 to 5).map { i => new Thread(() => { info(s"Starting submission in thread $i") try { From 15ea677be2a581f0a5380880ffc9c9d43ed7ec10 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 02:19:07 +0800 Subject: [PATCH 22/38] fix the case of dir which does not end with '/' --- .../connector/yarn/YarnLogPartitionReader.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 32a0aed9366..3ada2fc30b8 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -17,6 +17,11 @@ package org.apache.kyuubi.spark.connector.yarn +import java.io.{BufferedReader, InputStreamReader} + +import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.IOUtils @@ -25,10 +30,6 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String -import java.io.{BufferedReader, InputStreamReader} -import scala.collection.mutable.ArrayBuffer -import scala.util.matching.Regex - class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { @@ -74,7 +75,9 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) private def fetchLog(): Seq[LogEntry] = { val logDirInReg = yarnLogPartition.remoteAppLogDir match { // in case of /tmp/logs/, /tmp/logs// - case dir if dir.endsWith("/") => dir.replaceAll("/+", "/").replace("/", "\\/") + case dir if dir.endsWith("/") => + val tmpDir = dir.replaceAll("/+", "/") + tmpDir.substring(0, tmpDir.length - 1).replace("/", "\\/") // in case of /tmp/logs case dir => dir.replace("/", "\\/") } @@ -93,7 +96,6 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) line = reader.readLine(); line != null }) { - // println(s"Line $lineNumber: $line") lineNumber += 1 logEntries += LogEntry( applicationId, From 531d33ceb5a5e0eedc5b7a2e67220532615d2b6d Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 02:25:07 +0800 Subject: [PATCH 23/38] fix the case of predicate push down when query apps table with equalTo or in some specific state(s) --- .../connector/yarn/YarnAppPartitionReader.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index cd757c1014e..d5d1e14832d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,10 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient @@ -31,6 +27,10 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.sources.{EqualTo, In} import org.apache.spark.unsafe.types.UTF8String +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { @@ -79,7 +79,7 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) case EqualTo("id", appId: String) => java.util.Collections.singletonList( yarnClient.getApplicationReport(ApplicationId.fromString(appId))) case EqualTo("state", state: String) => - state.toUpperCase match { + state match { case validState if validYarnStateSet.contains(validState) => yarnClient.getApplications( java.util.EnumSet.of(YarnApplicationState.valueOf(validState))) @@ -89,7 +89,7 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) yarnClient.getApplications(java.util.Collections.singleton(appType)) case In("state", states: Array[Any]) => yarnClient.getApplications( java.util.EnumSet.copyOf(states - .map(x => x.toString.toUpperCase) + .map(x => x.toString) .filter(x => validYarnStateSet.contains(x)) .map(x => YarnApplicationState.valueOf(x)).toList.asJava)) From a2a2164562755998a043f6f7ec0bbcaef7f998d9 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 02:40:01 +0800 Subject: [PATCH 24/38] fix code style problem in YarnAppPartitionReader --- .../spark/connector/yarn/YarnAppPartitionReader.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index d5d1e14832d..6b06ce7252c 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -17,6 +17,10 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.jdk.CollectionConverters.asScalaBufferConverter + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient @@ -27,10 +31,6 @@ import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.sources.{EqualTo, In} import org.apache.spark.unsafe.types.UTF8String -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaBufferConverter - class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) extends PartitionReader[InternalRow] with Logging { From 1b46b7ba56c043ed321fc7e5fa1fc11cb839cc16 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 02:56:34 +0800 Subject: [PATCH 25/38] fix star match in log dir --- .../yarn/YarnLogPartitionReader.scala | 22 ++++++++----------- .../spark/connector/yarn/YarnLogScan.scala | 8 +++---- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 3ada2fc30b8..1a789c7a56c 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -17,28 +17,21 @@ package org.apache.kyuubi.spark.connector.yarn -import java.io.{BufferedReader, InputStreamReader} - -import scala.collection.mutable.ArrayBuffer -import scala.util.matching.Regex - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.io.IOUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String +import java.io.{BufferedReader, InputStreamReader} +import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex + class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { - private val fs = { - val hadoopConf = new Configuration() - yarnLogPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) - FileSystem.get(hadoopConf) - } - private val logIterator = fetchLog().iterator override def next(): Boolean = logIterator.hasNext @@ -86,6 +79,9 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) yarnLogPartition.logPath match { case pathPattern(user, applicationId, containerHost, containerSuffix) => val path = new Path(yarnLogPartition.logPath) + val hadoopConf = new Configuration() + yarnLogPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2)) + val fs = path.getFileSystem(hadoopConf) val inputStream = fs.open(path) val reader = new BufferedReader(new InputStreamReader(inputStream)) var line: String = null @@ -93,7 +89,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) val logEntries = new ArrayBuffer[LogEntry]() try { while ({ - line = reader.readLine(); + line = reader.readLine() line != null }) { lineNumber += 1 diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index abbb6291cfe..9a2b4d8256d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -91,13 +91,13 @@ case class YarnLogScan( case pushed if pushed.isEmpty => listFiles(remoteAppLogDir) case pushed => pushed.collectFirst { case EqualTo("app_id", appId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/*/${appId}") ++ + listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") ++ // compatible for hadoop2 - listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") + listFiles(s"${remoteAppLogDir}/*/*/${appId}") case EqualTo("container_id", containerId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/*/*/${containerId}") ++ + listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") ++ // compatible for hadoop2 - listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") + listFiles(s"${remoteAppLogDir}/*/*/*/${containerId}") case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") case _ => listFiles(remoteAppLogDir) }.get From c15055860e211258530d137269b956b0264f6b75 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 02:57:22 +0800 Subject: [PATCH 26/38] fix code style problem in YarnAppPartitionReader --- .../spark/connector/yarn/YarnLogPartitionReader.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index 1a789c7a56c..cf33c81eebb 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -17,6 +17,11 @@ package org.apache.kyuubi.spark.connector.yarn +import java.io.{BufferedReader, InputStreamReader} + +import scala.collection.mutable.ArrayBuffer +import scala.util.matching.Regex + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.IOUtils @@ -25,10 +30,6 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.unsafe.types.UTF8String -import java.io.{BufferedReader, InputStreamReader} -import scala.collection.mutable.ArrayBuffer -import scala.util.matching.Regex - class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) extends PartitionReader[InternalRow] { From 240393c5e7c51b42775fb9e558c71e38e44eb972 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 03:19:23 +0800 Subject: [PATCH 27/38] remove fs close --- .../kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index cf33c81eebb..a228c1c4320 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -49,9 +49,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) UTF8String.fromString(yarnLog.message))) } - override def close(): Unit = { - fs.close() - } + override def close(): Unit = {} /** * fet log @@ -107,6 +105,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) } finally { IOUtils.closeStream(inputStream) reader.close() + fs.close() } case _ => Seq.empty } From 5f6597a5da7763271683442faa61b10539959b09 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 03:26:26 +0800 Subject: [PATCH 28/38] remove supports for hadoop2 --- .../spark/connector/yarn/YarnLogPartitionReader.scala | 2 -- .../kyuubi/spark/connector/yarn/YarnLogScan.scala | 10 ++-------- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index a228c1c4320..bedb90f2bac 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -56,8 +56,6 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) * * hadoop3: * * /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 * * /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 - * * hadoop2: - * * /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 * * @param logStatus * @param user diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index 9a2b4d8256d..277992b80e5 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -81,8 +81,6 @@ case class YarnLogScan( * hadoop3: * /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 * /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 - * hadoop2: - * /tmp/logs/xxx/logs/application_1716268141594_240044/node10_35254 * * @return */ @@ -91,13 +89,9 @@ case class YarnLogScan( case pushed if pushed.isEmpty => listFiles(remoteAppLogDir) case pushed => pushed.collectFirst { case EqualTo("app_id", appId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") ++ - // compatible for hadoop2 - listFiles(s"${remoteAppLogDir}/*/*/${appId}") + listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") case EqualTo("container_id", containerId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") ++ - // compatible for hadoop2 - listFiles(s"${remoteAppLogDir}/*/*/*/${containerId}") + listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") case _ => listFiles(remoteAppLogDir) }.get From f9f1fb201e4484f37828c9be0f11c2d3476a1b8d Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 14:40:38 +0800 Subject: [PATCH 29/38] fix predicates push down error when query app_logs with line_num --- .../yarn/YarnAppPartitionReader.scala | 51 ++++++++++++------- .../connector/yarn/YarnAppScanBuilder.scala | 15 ++++++ .../connector/yarn/YarnLogPartition.scala | 4 +- .../yarn/YarnLogPartitionReader.scala | 3 +- .../spark/connector/yarn/YarnLogScan.scala | 10 ++-- .../connector/yarn/YarnLogScanBuilder.scala | 14 ++++- .../WithKyuubiServerAndYarnMiniCluster.scala | 6 +++ .../connector/yarn/YarnAppQuerySuite.scala | 1 - .../connector/yarn/YarnLogQuerySuite.scala | 10 ++-- 9 files changed, 84 insertions(+), 30 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 6b06ce7252c..936fb9b11bb 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -28,7 +28,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader -import org.apache.spark.sql.sources.{EqualTo, In} +import org.apache.spark.sql.sources.{EqualTo, Filter, In} import org.apache.spark.unsafe.types.UTF8String class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) @@ -98,26 +98,41 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) case _ => yarnClient.getApplications() }.get } - - val appSeq = applicationReports.asScala.map(app => { - YarnApplication( - id = app.getApplicationId.toString, - appType = app.getApplicationType, - user = app.getUser, - name = app.getName, - state = app.getYarnApplicationState.name, - queue = app.getQueue, - attemptId = app.getCurrentApplicationAttemptId.toString, - submitTime = app.getSubmitTime, - launchTime = app.getLaunchTime, - startTime = app.getStartTime, - finishTime = app.getFinishTime, - trackingUrl = app.getTrackingUrl, - originalTrackingUrl = app.getOriginalTrackingUrl) - }) + val appSeq = applicationReports.asScala.filter(app => + yarnAppPartition.filters + .forall(filter => maybeFilter(app, filter))) + .map(app => { + YarnApplication( + id = app.getApplicationId.toString, + appType = app.getApplicationType, + user = app.getUser, + name = app.getName, + state = app.getYarnApplicationState.name, + queue = app.getQueue, + attemptId = app.getCurrentApplicationAttemptId.toString, + submitTime = app.getSubmitTime, + launchTime = app.getLaunchTime, + startTime = app.getStartTime, + finishTime = app.getFinishTime, + trackingUrl = app.getTrackingUrl, + originalTrackingUrl = app.getOriginalTrackingUrl) + }) yarnClient.close() appSeq } + + private def maybeFilter(app: ApplicationReport, filter: Filter): Boolean = { + filter match { + case EqualTo("id", appId: String) => app.getApplicationId.toString eq appId + case EqualTo("state", appState: String) => app.getYarnApplicationState.name() eq appState + case EqualTo("type", appType: String) => app.getApplicationType eq appType + case In("state", states) => states.map(x => x.toString) + .contains(app.getYarnApplicationState.name()) + case In("type", types) => types.map(x => x.toString) + .contains(app.getApplicationType) + case _ => false + } + } } // Helper class to represent app diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala index 3af6112354b..cc256f5d64b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala @@ -18,6 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -27,4 +28,18 @@ case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructT override def build(): Scan = { YarnAppScan(options, schema, pushed) } + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supportedFilter, unsupportedFilter) = filters.partition { + case filter: EqualTo => + filter match { + case EqualTo("app_id", _) => true + case EqualTo("user", _) => true + case _ => false + } + case _ => false + } + pushed = supportedFilter + unsupportedFilter + } } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala index 7c9a0683693..918d774ab00 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartition.scala @@ -18,9 +18,11 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.sources.Filter case class YarnLogPartition( hadoopConfMap: Map[String, String], logPath: String, - remoteAppLogDir: String) + remoteAppLogDir: String, + filters: Array[Filter]) extends InputPartition diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index bedb90f2bac..fcdaafb15b3 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -96,14 +96,13 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) s"${containerHost}_${containerSuffix}", containerHost, lineNumber, - path.getName, + path.toUri.getPath, line) } logEntries } finally { IOUtils.closeStream(inputStream) reader.close() - fs.close() } case _ => Seq.empty } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index 277992b80e5..ddb074b6c5f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -89,9 +89,13 @@ case class YarnLogScan( case pushed if pushed.isEmpty => listFiles(remoteAppLogDir) case pushed => pushed.collectFirst { case EqualTo("app_id", appId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") + listFiles(s"${remoteAppLogDir}/*/*/*/${appId}") ++ + // compatible for hadoop2 + listFiles(s"${remoteAppLogDir}/*/*/${appId}") case EqualTo("container_id", containerId: String) => - listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") + listFiles(s"${remoteAppLogDir}/*/*/*/*/${containerId}") ++ + // compatible for hadoop2 + listFiles(s"${remoteAppLogDir}/*/*/*/${containerId}") case EqualTo("user", user: String) => listFiles(s"${remoteAppLogDir}/${user}") case _ => listFiles(remoteAppLogDir) }.get @@ -101,7 +105,7 @@ case class YarnLogScan( override def planInputPartitions(): Array[InputPartition] = { // get file nums and construct nums inputPartition tryPushDownPredicates().map(fileStatus => { - YarnLogPartition(hadoopConfMap, fileStatus.getPath.toString, remoteAppLogDir) + YarnLogPartition(hadoopConfMap, fileStatus.getPath.toString, remoteAppLogDir, filters) }).toArray } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala index 57c6679414c..708dbf8482a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.{EqualTo, Filter, In} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -31,7 +31,17 @@ case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructT override def pushFilters(filters: Array[Filter]): Array[Filter] = { val (supportedFilter, unsupportedFilter) = filters.partition { - case _: org.apache.spark.sql.sources.EqualTo => true + case filter: EqualTo => + filter match { + case EqualTo("id", _) => true + case EqualTo("state", _) => true + case EqualTo("type", _) => true + } + case filter: In => + filter match { + case In("state", _) => true + case In("type", _) => true + } case _ => false } pushed = supportedFilter diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala index 78c11badfc9..2e53eea1caf 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/WithKyuubiServerAndYarnMiniCluster.scala @@ -20,6 +20,8 @@ package org.apache.kyuubi.spark.connector.yarn import java.io.{File, FileWriter} import java.util.Collections +import scala.util.Random + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records.{ApplicationSubmissionContext, ContainerLaunchContext, Resource, YarnApplicationState} import org.apache.hadoop.yarn.client.api.YarnClient @@ -34,6 +36,8 @@ import org.apache.kyuubi.util.JavaUtils trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiServer { + private val taskTypeSet: Set[String] = Set("TYPE_1", "TYPE_2", "TYPE_3") + override protected val conf: KyuubiConf = new KyuubiConf(false) val kyuubiHome: String = JavaUtils.getCodeSourceLocation(getClass).split("extensions").head @@ -154,6 +158,8 @@ trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiS .getApplicationSubmissionContext.getApplicationId appContext.setApplicationId(applicationId) appContext.setApplicationName("TestApp") + // use random pickup + appContext.setApplicationType(taskTypeSet.toSeq(Random.nextInt(taskTypeSet.size))) // Set up container launch context (e.g., commands to execute) val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala index 1b79e3d15ee..6dfe4402885 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppQuerySuite.scala @@ -190,5 +190,4 @@ class YarnAppQuerySuite extends SparkYarnConnectorWithYarn { yarnClient.close() } } - } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index 0599e18dbb4..ee894501002 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -42,7 +42,9 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") val cnt = spark.sql( - "select count(1) from yarn.default.app_logs where host='localhost'").collect().head.getLong( + "select count(1) from yarn.default.app_logs " + + "where (host='localhost' or host like '%host') and " + + "app_id like '%application%'").collect().head.getLong( 0) assert(cnt > 0) } @@ -66,8 +68,10 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { }) withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => spark.sql("USE yarn") - val host = spark.sql( - "select * from yarn.default.app_logs limit 10").collect().head.getString(2) + val rows = spark.sql( + "select * from yarn.default.app_logs where line_num = 10" + + " limit 10").collect() + val host = rows.head.getString(2) assert(host == "localhost") } } From b2ccf504e15c3580871fb9fa3365a8f4e111e5d9 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 22:16:05 +0800 Subject: [PATCH 30/38] fix predicates push down error when query app_logs with line_num and make line_num starts from 1. --- .../yarn/YarnAppPartitionReader.scala | 6 ++--- .../connector/yarn/YarnAppScanBuilder.scala | 13 +++++++--- .../yarn/YarnLogPartitionReader.scala | 19 +++++++++++--- .../spark/connector/yarn/YarnLogScan.scala | 26 +++++++++---------- .../connector/yarn/YarnLogScanBuilder.scala | 14 ++++------ .../connector/yarn/YarnLogQuerySuite.scala | 1 + 6 files changed, 46 insertions(+), 33 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala index 936fb9b11bb..62bce64d46d 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala @@ -123,9 +123,9 @@ class YarnAppPartitionReader(yarnAppPartition: YarnAppPartition) private def maybeFilter(app: ApplicationReport, filter: Filter): Boolean = { filter match { - case EqualTo("id", appId: String) => app.getApplicationId.toString eq appId - case EqualTo("state", appState: String) => app.getYarnApplicationState.name() eq appState - case EqualTo("type", appType: String) => app.getApplicationType eq appType + case EqualTo("id", appId: String) => app.getApplicationId.toString.equals(appId) + case EqualTo("state", appState: String) => app.getYarnApplicationState.name().equals(appState) + case EqualTo("type", appType: String) => app.getApplicationType.equals(appType) case In("state", states) => states.map(x => x.toString) .contains(app.getYarnApplicationState.name()) case In("type", types) => types.map(x => x.toString) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala index cc256f5d64b..0ca7c5fefe9 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.{EqualTo, Filter} +import org.apache.spark.sql.sources.{EqualTo, Filter, In} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,8 +33,15 @@ case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructT val (supportedFilter, unsupportedFilter) = filters.partition { case filter: EqualTo => filter match { - case EqualTo("app_id", _) => true - case EqualTo("user", _) => true + case EqualTo("id", _) => true + case EqualTo("state", _) => true + case EqualTo("type", _) => true + case _ => false + } + case filter: In => + filter match { + case In("state", _) => true + case In("type", _) => true case _ => false } case _ => false diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala index fcdaafb15b3..3825888e0ff 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogPartitionReader.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.io.IOUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.unsafe.types.UTF8String class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) @@ -52,8 +53,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) override def close(): Unit = {} /** - * fet log - * * hadoop3: + * fetch log * * /tmp/logs/xxx/bucket-xxx-tfile/0001/application_1734531705578_0001/localhost_32422 * * /tmp/logs/xxx/bucket-logs-tfile/0001/application_1734530210878_0001/localhost_24232 * @@ -82,7 +82,7 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) val inputStream = fs.open(path) val reader = new BufferedReader(new InputStreamReader(inputStream)) var line: String = null - var lineNumber: Int = 1 + var lineNumber: Int = 0 val logEntries = new ArrayBuffer[LogEntry]() try { while ({ @@ -99,7 +99,9 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) path.toUri.getPath, line) } - logEntries + logEntries.filter(entry => + yarnLogPartition.filters.forall(filter => + maybeFilter(entry, filter))) } finally { IOUtils.closeStream(inputStream) reader.close() @@ -107,6 +109,15 @@ class YarnLogPartitionReader(yarnLogPartition: YarnLogPartition) case _ => Seq.empty } } + + private def maybeFilter(entry: LogEntry, filter: Filter): Boolean = { + filter match { + case EqualTo("app_id", appId: String) => entry.appId.equals(appId) + case EqualTo("container_id", containerId: String) => entry.containerId.equals(containerId) + case EqualTo("user", user: String) => entry.user.equals(user) + case _ => false + } + } } // Helper class to represent log entries diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index ddb074b6c5f..dd4fc37833f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -17,8 +17,6 @@ package org.apache.kyuubi.spark.connector.yarn -import scala.collection.mutable - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.sql.SparkSession @@ -27,6 +25,8 @@ import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap +import scala.collection.mutable + case class YarnLogScan( options: CaseInsensitiveStringMap, schema: StructType, @@ -58,18 +58,16 @@ case class YarnLogScan( val fs = FileSystem.get(hadoopConf) val path = new Path(pathStr) val logFiles = mutable.ArrayBuffer[FileStatus]() - if (fs.exists(path)) { - val fileStatuses: Array[FileStatus] = fs.globStatus(path) - if (fileStatuses != null && fileStatuses.nonEmpty) { - fileStatuses.foreach { - case status if status.isFile => logFiles += status - case status if status.isDirectory => - val fileIterator = fs.listFiles(status.getPath, true) - while (fileIterator.hasNext) { - val fileStatus = fileIterator.next() - if (fileStatus.isFile) logFiles += fileStatus - } - } + val fileStatuses: Array[FileStatus] = fs.globStatus(path) + if (fileStatuses != null && fileStatuses.nonEmpty) { + fileStatuses.foreach { + case status if status.isFile => logFiles += status + case status if status.isDirectory => + val fileIterator = fs.listFiles(status.getPath, true) + while (fileIterator.hasNext) { + val fileStatus = fileIterator.next() + if (fileStatus.isFile) logFiles += fileStatus + } } } fs.close() diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala index 708dbf8482a..7c17dd77502 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala @@ -18,7 +18,7 @@ package org.apache.kyuubi.spark.connector.yarn import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.{EqualTo, Filter, In} +import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,14 +33,10 @@ case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructT val (supportedFilter, unsupportedFilter) = filters.partition { case filter: EqualTo => filter match { - case EqualTo("id", _) => true - case EqualTo("state", _) => true - case EqualTo("type", _) => true - } - case filter: In => - filter match { - case In("state", _) => true - case In("type", _) => true + case EqualTo("app_id", _) => true + case EqualTo("user", _) => true + case EqualTo("container_id", _) => true + case _ => false } case _ => false } diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index ee894501002..42b065d6fad 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -70,6 +70,7 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { spark.sql("USE yarn") val rows = spark.sql( "select * from yarn.default.app_logs where line_num = 10" + + " and user='zhangxinsen'" + " limit 10").collect() val host = rows.head.getString(2) assert(host == "localhost") From c1775eecdb3731969ed3742525aa7fdfc9fd6872 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 22:18:27 +0800 Subject: [PATCH 31/38] fix style check error --- .../org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala index dd4fc37833f..8906784636f 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala @@ -17,6 +17,8 @@ package org.apache.kyuubi.spark.connector.yarn +import scala.collection.mutable + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.sql.SparkSession @@ -25,8 +27,6 @@ import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -import scala.collection.mutable - case class YarnLogScan( options: CaseInsensitiveStringMap, schema: StructType, From 21973e15f13606ae8b0096300e9775ac1ee70b88 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sat, 21 Dec 2024 22:33:57 +0800 Subject: [PATCH 32/38] change query condition from user into host and remove local username --- .../apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala index 42b065d6fad..96c3b1dea14 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/test/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogQuerySuite.scala @@ -70,7 +70,7 @@ class YarnLogQuerySuite extends SparkYarnConnectorWithYarn { spark.sql("USE yarn") val rows = spark.sql( "select * from yarn.default.app_logs where line_num = 10" + - " and user='zhangxinsen'" + + " and host='localhost'" + " limit 10").collect() val host = rows.head.getString(2) assert(host == "localhost") From e0f0bbfc8afbd0e2691dea522f1f226f26c62cbc Mon Sep 17 00:00:00 2001 From: naive-zhang Date: Sun, 22 Dec 2024 11:39:33 +0800 Subject: [PATCH 33/38] Update extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala Co-authored-by: Cheng Pan --- .../apache/kyuubi/spark/connector/yarn/YarnLogTable.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala index dc9fa3899c9..45607e2074b 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala @@ -41,8 +41,6 @@ class YarnLogTable extends Table with SupportsRead { override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava - override def newScanBuilder(caseInsensitiveStringMap: CaseInsensitiveStringMap): ScanBuilder = - YarnLogScanBuilder( - caseInsensitiveStringMap, - schema()) + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = + YarnLogScanBuilder(options,schema()) } From 9be34a7c5dbf9c65303f4ce7063b27bb2e1f88af Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sun, 22 Dec 2024 11:40:48 +0800 Subject: [PATCH 34/38] canonicalize some error message --- .../spark/connector/yarn/YarnCatalog.scala | 44 ++++--------------- 1 file changed, 9 insertions(+), 35 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala index a6f6f742517..2c68bda0548 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnCatalog.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { +class YarnCatalog extends TableCatalog with Logging { private var catalogName: String = _ override def initialize( @@ -53,50 +53,24 @@ class YarnCatalog extends TableCatalog with SupportsNamespaces with Logging { structType: StructType, transforms: Array[Transform], map: util.Map[String, String]): Table = { - throw new UnsupportedOperationException("Create table is not supported") + throw new UnsupportedOperationException(s"The tables in catalog " + + s"${catalogName} does not support CREATE TABLE") } override def alterTable(identifier: Identifier, tableChanges: TableChange*): Table = { - throw new UnsupportedOperationException("Alter table is not supported") + throw new UnsupportedOperationException(s"The tables in catalog " + + s"${catalogName} does not support ALTER TABLE") } override def dropTable(identifier: Identifier): Boolean = { - throw new UnsupportedOperationException("Drop table is not supported") + throw new UnsupportedOperationException(s"The tables in catalog " + + s"${catalogName} does not support DROP TABLE") } override def renameTable(identifier: Identifier, identifier1: Identifier): Unit = { - throw new UnsupportedOperationException("Rename table is not supported") + throw new UnsupportedOperationException(s"The tables in catalog " + + s"${catalogName} does not support RENAME TABLE") } - override def listNamespaces(): Array[Array[String]] = { - Array(Array("default")) - } - - override def listNamespaces(namespace: Array[String]): Array[Array[String]] = namespace match { - case Array() => listNamespaces() - case Array(db) if db eq "default" => listNamespaces() - case _ => throw new NoSuchNamespaceException(namespace) - } - - override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = - namespace match { - case Array(_) => Map.empty[String, String].asJava - case _ => throw new NoSuchNamespaceException(namespace) - } - - override def createNamespace(namespace: Array[String], metadata: util.Map[String, String]): Unit = - throw new UnsupportedOperationException - - override def alterNamespace(namespace: Array[String], changes: NamespaceChange*): Unit = - throw new UnsupportedOperationException - - // Removed in SPARK-37929 - def dropNamespace(namespace: Array[String]): Boolean = - throw new UnsupportedOperationException - - // Introduced in SPARK-37929 - def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean = - throw new UnsupportedOperationException - override def name(): String = this.catalogName } From 54e1bafe54a49437ededc660b462a6f67e8e547d Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sun, 22 Dec 2024 11:44:37 +0800 Subject: [PATCH 35/38] remove unnecessary abstract class BasicScanBuilder --- .../connector/yarn/BasicScanBuilder.scala | 40 ------------------- .../connector/yarn/YarnAppScanBuilder.scala | 13 +++--- .../connector/yarn/YarnLogScanBuilder.scala | 13 +++--- 3 files changed, 14 insertions(+), 52 deletions(-) delete mode 100644 extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala deleted file mode 100644 index 862716f81b9..00000000000 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/BasicScanBuilder.scala +++ /dev/null @@ -1,40 +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.kyuubi.spark.connector.yarn - -import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.sources.Filter - -trait BasicScanBuilder - extends ScanBuilder - with SupportsPushDownFilters with Serializable { - - protected var pushed: Array[Filter] = Array.empty - - override def pushFilters(filters: Array[Filter]): Array[Filter] = { - val (supportedFilter, unsupportedFilter) = filters.partition { - case _: org.apache.spark.sql.sources.EqualTo => true - case _: org.apache.spark.sql.sources.In => true - case _ => false - } - pushed = supportedFilter - unsupportedFilter - } - - override def pushedFilters(): Array[Filter] = pushed -} diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala index 0ca7c5fefe9..59d93c989d0 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala @@ -22,14 +22,15 @@ import org.apache.spark.sql.sources.{EqualTo, Filter, In} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) - extends BasicScanBuilder { +case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) { - override def build(): Scan = { - YarnAppScan(options, schema, pushed) - } + private var pushed: Array[Filter] = Array.empty + + def pushedFilters(): Array[Filter] = pushed + + def build(): Scan = YarnAppScan(options, schema, pushed) - override def pushFilters(filters: Array[Filter]): Array[Filter] = { + def pushFilters(filters: Array[Filter]): Array[Filter] = { val (supportedFilter, unsupportedFilter) = filters.partition { case filter: EqualTo => filter match { diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala index 7c17dd77502..cdd32beacd4 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala @@ -22,14 +22,15 @@ import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) - extends BasicScanBuilder { +case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) { - override def build(): Scan = { - YarnLogScan(options, schema, pushed) - } + private var pushed: Array[Filter] = Array.empty + + def pushedFilters(): Array[Filter] = pushed + + def build(): Scan = YarnAppScan(options, schema, pushed) - override def pushFilters(filters: Array[Filter]): Array[Filter] = { + def pushFilters(filters: Array[Filter]): Array[Filter] = { val (supportedFilter, unsupportedFilter) = filters.partition { case filter: EqualTo => filter match { From 2f171e8bed25b129bae6caa515c0cc3133f4fd82 Mon Sep 17 00:00:00 2001 From: naive-zhang Date: Sun, 22 Dec 2024 11:48:55 +0800 Subject: [PATCH 36/38] Update extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala Co-authored-by: Cheng Pan --- .../kyuubi/spark/connector/yarn/YarnApplicationTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala index 52a22d150bf..7df2a2468a0 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class YarnApplicationTable extends Table with SupportsRead { +class YarnAppTable extends Table with SupportsRead { override def name(): String = "apps" override def schema(): StructType = From b258a75e5882650ab3fd34e3fbd7ef013ea69124 Mon Sep 17 00:00:00 2001 From: naive-zhang Date: Sun, 22 Dec 2024 11:49:08 +0800 Subject: [PATCH 37/38] Update extensions/spark/kyuubi-spark-connector-yarn/pom.xml Co-authored-by: Cheng Pan --- extensions/spark/kyuubi-spark-connector-yarn/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml index 8b5e3c43694..24020a6d071 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/pom.xml +++ b/extensions/spark/kyuubi-spark-connector-yarn/pom.xml @@ -27,7 +27,7 @@ kyuubi-spark-connector-yarn_${scala.binary.version} jar - Kyuubi Spark Yarn Logs Connector + Kyuubi Spark Hadoop YARN Connector https://kyuubi.apache.org/ From 86f9476b1ccffd36475ef7b73f4d572d9f4f36e4 Mon Sep 17 00:00:00 2001 From: native-zhang Date: Sun, 22 Dec 2024 12:13:34 +0800 Subject: [PATCH 38/38] remove unnecessary abstract class BasicScanBuilder --- .../kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala | 3 ++- .../kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala index 59d93c989d0..0e52995ca4a 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScanBuilder.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.sources.{EqualTo, Filter, In} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) { +case class YarnAppScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) + extends ScanBuilder { private var pushed: Array[Filter] = Array.empty diff --git a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala index cdd32beacd4..6698a2c1520 100644 --- a/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala +++ b/extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScanBuilder.scala @@ -22,13 +22,14 @@ import org.apache.spark.sql.sources.{EqualTo, Filter} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) { +case class YarnLogScanBuilder(options: CaseInsensitiveStringMap, schema: StructType) + extends ScanBuilder { private var pushed: Array[Filter] = Array.empty def pushedFilters(): Array[Filter] = pushed - def build(): Scan = YarnAppScan(options, schema, pushed) + override def build(): Scan = YarnAppScan(options, schema, pushed) def pushFilters(filters: Array[Filter]): Array[Filter] = { val (supportedFilter, unsupportedFilter) = filters.partition {