pan3793 commented on code in PR #6856:
URL: https://github.com/apache/kyuubi/pull/6856#discussion_r1894650315


##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.{IntegerType, StringType, StructField, 
StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class YarnLogTable extends Table with SupportsRead {
+  override def name(): String = "app_logs"
+
+  override def schema(): StructType =

Review Comment:
   Spark also has `SupportsMetadataColumns`, maybe we should consider 
converting some cols to metadata col



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.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, Filter, In}
+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
+
+  override def get(): InternalRow = {
+    val app = appIterator.next()
+    new GenericInternalRow(Array[Any](
+      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,
+      app.finishTime,
+      UTF8String.fromString(app.trackingUrl),
+      UTF8String.fromString(app.originalTrackingUrl)))
+  }
+
+  override def close(): Unit = {}
+
+  private def fetchApp(): mutable.Seq[YarnApplication] = {
+    val hadoopConf = new Configuration()
+    yarnAppPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2))
+    val yarnClient = YarnClient.createYarnClient()
+    yarnClient.init(hadoopConf)
+    yarnClient.start()
+    // fet apps
+    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(

Review Comment:
   one application may have multiple attempts



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogTable.scala:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.{IntegerType, StringType, StructField, 
StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class YarnLogTable extends Table with SupportsRead {
+  override def name(): String = "app_logs"
+
+  override def schema(): StructType =
+    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("line_num", IntegerType, nullable = false),
+      StructField("file_name", 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 =
+    YarnLogScanBuilder(
+      caseInsensitiveStringMap,
+      schema())

Review Comment:
   ```suggestion
     override def newScanBuilder(options: CaseInsensitiveStringMap): 
ScanBuilder =
       YarnLogScanBuilder(options,schema())
   ```



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.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, Filter, In}
+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")

Review Comment:
   move the constant to the `object YarnAppPartitionReader` and leave a comment 
to explain where the list comes from



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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),
+      StructField("tracking_url", StringType, nullable = false),
+      StructField("original_tracking_url", StringType, nullable = false)))

Review Comment:
   I'm not sure the `tracking_url` is always present



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.{EqualTo, 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])
+  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]()
+    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
+   *
+   * @return
+   */
+  private def tryPushDownPredicates(): mutable.Seq[FileStatus] = {
+    filters match {
+      case pushed if pushed.isEmpty => listFiles(remoteAppLogDir)
+      case pushed => pushed.collectFirst {

Review Comment:
   it should support multiple predications ...



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.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, Filter, In}
+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
+
+  override def get(): InternalRow = {
+    val app = appIterator.next()
+    new GenericInternalRow(Array[Any](
+      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,
+      app.finishTime,
+      UTF8String.fromString(app.trackingUrl),
+      UTF8String.fromString(app.originalTrackingUrl)))
+  }
+
+  override def close(): Unit = {}
+
+  private def fetchApp(): mutable.Seq[YarnApplication] = {
+    val hadoopConf = new Configuration()
+    yarnAppPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2))
+    val yarnClient = YarnClient.createYarnClient()
+    yarnClient.init(hadoopConf)
+    yarnClient.start()
+    // fet apps
+    val applicationReports: java.util.List[ApplicationReport] =
+      yarnAppPartition.filters match {
+        case filters if filters.isEmpty => yarnClient.getApplications

Review Comment:
   will it retrieve all apps into memory, or streamly?



##########
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

Review Comment:
   I don't think this abstract layer is really helpful



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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 {

Review Comment:
   ```suggestion
   class YarnAppTable extends Table with SupportsRead {
   ```



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppPartitionReader.scala:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.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, Filter, In}
+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
+
+  override def get(): InternalRow = {
+    val app = appIterator.next()
+    new GenericInternalRow(Array[Any](
+      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,
+      app.finishTime,
+      UTF8String.fromString(app.trackingUrl),
+      UTF8String.fromString(app.originalTrackingUrl)))
+  }
+
+  override def close(): Unit = {}
+
+  private def fetchApp(): mutable.Seq[YarnApplication] = {
+    val hadoopConf = new Configuration()
+    yarnAppPartition.hadoopConfMap.foreach(kv => hadoopConf.set(kv._1, kv._2))
+    val yarnClient = YarnClient.createYarnClient()
+    yarnClient.init(hadoopConf)
+    yarnClient.start()
+    // fet apps
+    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) =>
+              state match {
+                case validState if validYarnStateSet.contains(validState) =>
+                  yarnClient.getApplications(
+                    
java.util.EnumSet.of(YarnApplicationState.valueOf(validState)))
+                case _ => Seq.empty[ApplicationReport].asJava
+              }
+            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 => x.toString)
+                  .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()
+          }.get
+      }
+    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.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)
+          .contains(app.getApplicationType)
+      case _ => false
+    }
+  }
+}
+
+// Helper class to represent app
+case class YarnApplication(

Review Comment:
   the class is not very helpful but would significantly increase memory 
consumption, let's convert the response data to InternelRow directly



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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),

Review Comment:
   why not TIMESTAMP?



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnAppScan.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 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 BasicScan {
+
+  // override def toBatch: Batch = this
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    Array(YarnAppPartition(

Review Comment:
   this actually hardcode a single partition, some thoughts:
   1. what kind of effective range conditions can be pushed to the YARN app 
list API? submit time range? app id range?
   2. does yarn provide a cheap API to know the earliest app submit time or app 
id?



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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),

Review Comment:
   what if app is not launched, started, finished?



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnApplicationTable.scala:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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),

Review Comment:
   put `attempt_id` after `id`



##########
extensions/spark/kyuubi-spark-connector-yarn/pom.xml:
##########
@@ -0,0 +1,275 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.kyuubi</groupId>
+        <artifactId>kyuubi-parent</artifactId>
+        <version>1.11.0-SNAPSHOT</version>
+        <relativePath>../../../pom.xml</relativePath>
+    </parent>
+
+    
<artifactId>kyuubi-spark-connector-yarn_${scala.binary.version}</artifactId>
+    <packaging>jar</packaging>
+    <name>Kyuubi Spark Yarn Logs Connector</name>

Review Comment:
   the connector is not limited to read YARN logs
   ```suggestion
       <name>Kyuubi Spark Hadoop YARN Connector</name>
   ```



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.{EqualTo, 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])
+  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]()
+    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
+   *
+   * @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}")

Review Comment:
   could you leave some comments to explain the directory structure and Hadoop 
code/JIRA reference?



##########
extensions/spark/kyuubi-spark-connector-yarn/src/main/scala/org/apache/kyuubi/spark/connector/yarn/YarnLogScan.scala:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.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.{EqualTo, 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])
+  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 {

Review Comment:
   this is incorrect.
   
   use `SparkSession.conf` in SQL cases instead of `SparkContext.getConf` in 
SQL cases because the latter returns the global static conf, which is immutable 
after Spark is launched.
   
   and I think it is not worth having an additional spark conf key, just use 
`SparkSession.sessionState.newHadoopConf` to create a `Configuration`, and read 
the hadoop conf directly.
   
   



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to