From 1af603b239fdf499796f6757aa3d7ebcbfa7f283 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 25 May 2026 15:58:29 +0530 Subject: [PATCH 01/22] K8s: Add AutoScaling --- packaging/src/kubernetes/README.md | 473 +++++++++++++++++- .../crds/hiveclusters.hive.apache.org-v1.yml | 152 ++++++ .../hive-operator/templates/clusterrole.yaml | 12 + .../hive-operator/templates/hivecluster.yaml | 36 ++ .../kubernetes/helm/hive-operator/values.yaml | 41 ++ .../dependent/HiveDependentResource.java | 139 +++++ .../HiveGenericDependentResource.java | 73 +++ .../HiveServer2DeploymentDependent.java | 104 +++- .../HiveServer2HttpScaledObjectDependent.java | 129 +++++ .../dependent/HiveServer2PdbDependent.java | 62 +++ .../HiveServer2ScaledObjectDependent.java | 149 ++++++ .../HiveServer2ServiceDependent.java | 8 + .../operator/dependent/LlapPdbDependent.java | 62 +++ .../dependent/LlapScaledObjectDependent.java | 158 ++++++ .../dependent/LlapStatefulSetDependent.java | 96 +++- .../MetastoreDeploymentDependent.java | 87 +++- .../dependent/MetastorePdbDependent.java | 62 +++ .../MetastoreScaledObjectDependent.java | 152 ++++++ .../operator/dependent/TezAmPdbDependent.java | 61 +++ .../dependent/TezAmScaledObjectDependent.java | 200 ++++++++ .../dependent/TezAmStatefulSetDependent.java | 73 ++- .../HiveServer2AutoscalingCondition.java | 41 ++ .../HiveServer2MetricScalingCondition.java | 44 ++ .../condition/HiveServer2Precondition.java | 9 +- .../HiveServer2ScaleToZeroCondition.java | 44 ++ .../condition/LlapAutoscalingCondition.java | 42 ++ .../MetastoreAutoscalingCondition.java | 42 ++ .../condition/MetastoreReadyCondition.java | 9 +- .../condition/TezAmAutoscalingCondition.java | 42 ++ .../operator/model/spec/AutoscalingSpec.java | 62 +++ .../operator/model/spec/HiveServer2Spec.java | 6 +- .../operator/model/spec/LlapSpec.java | 6 +- .../operator/model/spec/MetastoreSpec.java | 6 +- .../operator/model/spec/TezAmSpec.java | 6 +- .../reconciler/HiveClusterReconciler.java | 63 ++- .../kubernetes/operator/util/ConfigUtils.java | 20 + .../operator/util/HiveConfigBuilder.java | 22 + 37 files changed, 2745 insertions(+), 48 deletions(-) create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index 1fc11623240c..e9ccc62d949a 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -505,19 +505,449 @@ kubectl get hiveclusters kubectl describe hivecluster hive ``` +--- + +## Autoscaling + +The operator supports metric-based autoscaling for all four Hive components using +[KEDA](https://keda.sh/) ScaledObjects and Kubernetes-native HPA. Autoscaling is +opt-in per component and designed for **zero query failures** during scale-down. + +### Prerequisites + +- [KEDA](https://keda.sh/) installed in the cluster +- [Prometheus](https://prometheus.io/) scraping Hive pod metrics (for HS2, HMS, LLAP custom metrics) +- Kubernetes metrics-server (for CPU-based triggers on Tez AM) +- [KEDA HTTP Add-on](https://github.com/kedacore/http-add-on) — **required for `minReplicas: 0`**, enables automatic wake-from-zero for HS2 + +### Installing KEDA + +KEDA must be installed **before** enabling autoscaling on any Hive component. +The operator creates KEDA `ScaledObject` custom resources which require the KEDA +CRDs to be present on the cluster. + +```bash +# Add the KEDA Helm repo +helm repo add kedacore https://kedacore.github.io/charts +helm install keda kedacore/keda --namespace keda --create-namespace --wait +``` + +Verify KEDA is running: + +```bash +kubectl get pods -n keda +# Expected: keda-operator, keda-metrics-apiserver, keda-admission-webhooks +kubectl get crd | grep keda +# Expected: scaledobjects.keda.sh, scaledjobs.keda.sh, triggerauthentications.keda.sh, etc. +``` + +**For HS2 scale-to-zero** (`minReplicas: 0`), install the KEDA HTTP Add-on: + +```bash +helm install http-add-on kedacore/keda-add-ons-http \ + --namespace keda --wait +``` + +Verify the interceptor is running: + +```bash +kubectl get pods -n keda -l app=keda-add-ons-http-interceptor-proxy +# Expected: keda-add-ons-http-interceptor-proxy-... Running +``` + +> **Note:** The HTTP Add-on is required when `minReplicas: 0`. It places an interceptor +> proxy in the traffic path that detects incoming requests when HS2 has zero pods, +> automatically scaling HS2 up and holding the request until a pod is ready. + +**For Prometheus-based triggers** (HS2, HMS, LLAP), install Prometheus: + +```bash +helm repo add prometheus-community https://prometheus-community.github.io/helm-charts +helm install prometheus prometheus-community/prometheus \ + --namespace monitoring --create-namespace --wait +``` + +> **Note:** If autoscaling is enabled in the HiveCluster spec but KEDA is not +> installed, the operator will fail to reconcile with errors like +> `"Could not find the metadata for the given apiVersion and kind"`. +> Always install KEDA before setting `autoscaling.enabled: true`. + +### Graceful Scale-Down Architecture + +``` +┌─────────────────────────────────────────────────────────────────────┐ +│ Scale Down Flow │ +├─────────────────────────────────────────────────────────────────────┤ +│ 1. KEDA reduces desired replicas (cooldown elapsed, metric below │ +│ threshold) │ +│ 2. PodDisruptionBudget ensures minAvailable=1 (at least one pod │ +│ always running) │ +│ 3. Kubernetes sends SIGTERM to selected pod │ +│ 4. preStop hook runs: │ +│ - HS2: deregisters from ZK, drains open sessions │ +│ - HMS: sleeps 30s for in-flight Thrift RPCs │ +│ - LLAP: waits until all executors become idle │ +│ - TezAM: waits for current DAG completion │ +│ 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety net) │ +│ 6. Pod terminates only after drain completes │ +└─────────────────────────────────────────────────────────────────────┘ +``` + +### Per-Component Scaling Logic + +| Component | Scale-Up Trigger | Scale-Down Trigger | Cooldown | Native Metric | +|-----------|-----------------|-------------------|----------|---------------| +| **HiveServer2** | `hs2_active_sessions` > 80% of max **OR** CPU > 75% | `hs2_open_sessions` < 20% **AND** CPU < 30% | 10 min | `hs2_open_sessions`, `hs2_active_sessions` | +| **Metastore** | `api_get_partitions` rate spike **OR** CPU > 75% (2 min) | CPU < 30% **AND** API rate flat | 5 min | `api_get_partitions`, `open_connections` | +| **LLAP** | `NumQueuedRequests` > 0 for 1 min | `NumExecutorsAvailable == NumExecutors` (idle) | 15 min | `NumQueuedRequests`, `NumExecutorsAvailable` | +| **Tez AM** (with CPU resources) | Pod CPU > 60% (pool is busy) | Pod CPU < 10% (pool is idle) | 10 min | Standard K8s CPU | +| **Tez AM** (without CPU resources) | `tez_session_pending_tasks` > threshold | No pending tasks for cooldown | 10 min | `tez_session_pending_tasks` | + +### Scale-to-Zero Architecture + +When `minReplicas: 0` is configured (default for HS2, LLAP, TezAM), the cluster +scales down to zero pods when completely idle: + +``` + Scale-to-Zero (Idle Detection) + + 1. No active sessions/queries for cooldownPeriod seconds + → KEDA detects all triggers inactive + → scales HS2 to 0 (idleReplicaCount) + + 2. LLAP/TezAM ScaledObjects see hs2_open_sessions = 0 + → activation triggers inactive for cooldownPeriod + → scale LLAP and TezAM to 0 + + 3. HMS stays at minReplicas=1 (always available) + +``` + +``` + Wake-from-Zero (with KEDA HTTP Add-on) + + 1. Beeline connects → KEDA HTTP interceptor proxy queues the + request and triggers HS2 scale-up (0 → 1) + + 2. HS2 pod starts, reports hs2_open_sessions > 0 to Prometheus + + 3. KEDA detects cross-component activation trigger: + - LLAP ScaledObject sees hs2_open_sessions > 0 → scales up + - TezAM ScaledObject sees hs2_open_sessions > 0 → scales up + + 4. Query executes once LLAP/TezAM pods are ready + +``` + +> **Important:** Automatic wake-from-zero requires the KEDA HTTP Add-on. Traffic +> must flow through the interceptor proxy (via Ingress or port-forward). Without the +> HTTP Add-on, HS2 must be manually woken (`kubectl scale deployment/hive-hiveserver2 --replicas=1`). +> LLAP and TezAM wake automatically once HS2 reports active sessions. See +> [Connect to HiveServer2 > Connecting with Scale-to-Zero](#connecting-with-scale-to-zero-minreplicas--0) +> for setup instructions. + +**Component-specific behavior:** + +| Component | minReplicas | Scale-to-Zero Trigger | Wake Trigger | +|-----------|-------------|----------------------|--------------| +| **HS2** | 0 | `hs2_active_sessions = 0` for cooldown | HTTP request via KEDA interceptor (or manual) | +| **HMS** | 1 | Never (always running) | N/A | +| **LLAP** | 0 | `hs2_open_sessions = 0` for cooldown | `hs2_open_sessions > 0` (cross-component) | +| **TezAM** | 0 | `hs2_open_sessions = 0` + no pending tasks | `hs2_open_sessions > 0` (cross-component) | + +### Enabling Autoscaling + +**CLI (with Ozone storage backend):** + +```bash +helm install hive ./helm/hive-operator \ + --set cluster.database.type=postgres \ + --set cluster.database.url="jdbc:postgresql://postgres-postgresql:5432/metastore" \ + --set cluster.database.driver="org.postgresql.Driver" \ + --set cluster.database.username=hive \ + --set cluster.database.passwordSecretRef.name=hive-db-secret \ + --set cluster.database.passwordSecretRef.key=password \ + --set cluster.database.driverJarUrl="https://repo1.maven.org/maven2/org/postgresql/postgresql/42.7.5/postgresql-42.7.5.jar" \ + --set cluster.zookeeper.quorum="zookeeper:2181" \ + --set cluster.storage.coreSiteOverrides."fs\.defaultFS"="s3a://hive" \ + --set cluster.storage.coreSiteOverrides."fs\.s3a\.endpoint"="http://ozone-s3g-rest:9878" \ + --set-string cluster.storage.coreSiteOverrides."fs\.s3a\.path\.style\.access"=true \ + --set 'cluster.storage.envVars[0].name=HADOOP_OPTIONAL_TOOLS' \ + --set 'cluster.storage.envVars[0].value=hadoop-aws' \ + --set 'cluster.storage.envVars[1].name=AWS_ACCESS_KEY_ID' \ + --set 'cluster.storage.envVars[1].value=ozone' \ + --set 'cluster.storage.envVars[2].name=AWS_SECRET_ACCESS_KEY' \ + --set 'cluster.storage.envVars[2].value=ozone' \ + --set cluster.hiveServer2.autoscaling.enabled=true \ + --set cluster.hiveServer2.autoscaling.minReplicas=0 \ + --set cluster.hiveServer2.autoscaling.scaleUpThreshold=80 \ + --set cluster.hiveServer2.autoscaling.cooldownSeconds=600 \ + --set cluster.hiveServer2.autoscaling.gracePeriodSeconds=300 \ + --set cluster.metastore.autoscaling.enabled=true \ + --set cluster.metastore.autoscaling.minReplicas=1 \ + --set cluster.metastore.autoscaling.cooldownSeconds=300 \ + --set cluster.metastore.autoscaling.gracePeriodSeconds=60 \ + --set cluster.llap.autoscaling.enabled=true \ + --set cluster.llap.autoscaling.minReplicas=0 \ + --set cluster.llap.autoscaling.cooldownSeconds=900 \ + --set cluster.llap.autoscaling.gracePeriodSeconds=600 \ + --set cluster.tezAm.autoscaling.enabled=true \ + --set cluster.tezAm.autoscaling.minReplicas=0 \ + --set cluster.tezAm.autoscaling.scaleUpThreshold=5 \ + --set cluster.tezAm.autoscaling.cooldownSeconds=600 \ + --set cluster.tezAm.autoscaling.gracePeriodSeconds=120 +``` + +**Values file:** + +```yaml +# values-autoscaling.yaml +cluster: + database: + type: postgres + url: "jdbc:postgresql://postgres-postgresql:5432/metastore" + driver: "org.postgresql.Driver" + username: hive + passwordSecretRef: + name: hive-db-secret + key: password + driverJarUrl: "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.7.5/postgresql-42.7.5.jar" + + zookeeper: + quorum: "zookeeper:2181" + + storage: + coreSiteOverrides: + fs.defaultFS: "s3a://hive" + fs.s3a.endpoint: "http://ozone-s3g-rest:9878" + fs.s3a.path.style.access: "true" + envVars: + - name: HADOOP_OPTIONAL_TOOLS + value: "hadoop-aws" + - name: AWS_ACCESS_KEY_ID + value: "ozone" + - name: AWS_SECRET_ACCESS_KEY + value: "ozone" + + hiveServer2: + replicas: 10 # Acts as max replicas when autoscaling is enabled + resources: + requestsCpu: "1" # Required for CPU-based autoscaling trigger + requestsMemory: "2Gi" + autoscaling: + enabled: true + minReplicas: 0 # Scale to zero when idle + scaleUpThreshold: 80 # Requests/sec that triggers additional pods + cooldownSeconds: 600 # 10 min before scaling back to 0 + gracePeriodSeconds: 300 + + metastore: + replicas: 6 # Acts as max replicas when autoscaling is enabled + resources: + requestsCpu: "500m" # Required for CPU-based autoscaling trigger + requestsMemory: "1Gi" + autoscaling: + enabled: true + minReplicas: 1 # HMS must always be available + cooldownSeconds: 300 + gracePeriodSeconds: 60 + + llap: + replicas: 8 # Acts as max replicas when autoscaling is enabled + autoscaling: + enabled: true + minReplicas: 0 # Scale to zero when no queries need LLAP + cooldownSeconds: 900 # 15 min — scaling down destroys in-memory cache + gracePeriodSeconds: 600 + + tezAm: + replicas: 10 # Acts as max replicas when autoscaling is enabled + resources: + requestsCpu: "500m" # Required for CPU-based autoscaling trigger + requestsMemory: "1Gi" + autoscaling: + enabled: true + minReplicas: 0 # Scale to zero when no queries running + scaleUpThreshold: 60 # CPU% when resources set; pending tasks per AM otherwise + scaleDownThreshold: 10 + cooldownSeconds: 600 + gracePeriodSeconds: 120 +``` + +```bash +helm install hive ./helm/hive-operator -f values-autoscaling.yaml +``` + +When autoscaling is enabled, the operator automatically: +- Deploys the Prometheus JMX Exporter agent sidecar (port 9404, `/metrics`) +- Enables `hive.server2.metrics.enabled` / `metastore.metrics.enabled` (JMX reporter) +- Adds Prometheus scrape annotations to pods +- Creates KEDA ScaledObjects with the configured thresholds +- Creates PodDisruptionBudgets (minAvailable: 1) +- Configures preStop lifecycle hooks for graceful drain +- Sets `terminationGracePeriodSeconds` to the configured grace period +- Adds cross-component activation triggers for LLAP/TezAM (wake when HS2 has open sessions) + +**Exported Prometheus Metrics (per component):** + +| Component | Metrics | Purpose | +|-----------|---------|---------| +| **HiveServer2** | `hs2_open_sessions`, `hs2_active_sessions`, `hs2_active_calls_*`, `tez_session_pending_tasks`, `tez_session_running_tasks`, `tez_session_task_backlog_ratio` | Session/query load, Tez AM demand | +| **Metastore** | `api_*_total`, `hive_metastore_open_connections` | API call rates, connection count | +| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_*` | Executor queue depth, daemon health | +| **Tez AM** | `tez_am_*` | DAG execution metrics | + +### CPU-Based Scaling and Resource Requests + +The operator includes a **CPU utilization trigger** in the ScaledObject for HS2, Metastore, +and Tez AM. KEDA's CPU trigger uses the `Utilization` metric type, which is defined as a +percentage of the container's CPU request. This means **the container must have a CPU request +defined** for the trigger to work. + +If you enable autoscaling without setting `resources` for that component, the operator +will omit the CPU trigger and rely solely on the Prometheus-based trigger. For Tez AM +specifically, without CPU resources the operator uses `tez_session_pending_tasks` (queued +tasks waiting for AM slots) as the proportional scaler — this reflects real query demand +rather than connection count, avoiding spurious scale-ups from idle or zombie sessions. + +To get both Prometheus and CPU-based scaling, set `resources` on the component: + +```yaml +cluster: + hiveServer2: + resources: + requestsCpu: "1" # Required for CPU-based autoscaling + requestsMemory: "2Gi" + autoscaling: + enabled: true + + metastore: + resources: + requestsCpu: "500m" # Required for CPU-based autoscaling + requestsMemory: "1Gi" + autoscaling: + enabled: true + + tezAm: + resources: + requestsCpu: "500m" # Required for CPU-based autoscaling + requestsMemory: "1Gi" + autoscaling: + enabled: true +``` + +> **Note:** LLAP scaling uses only Prometheus triggers (`NumQueuedRequests`) +> and does not include a CPU trigger, so LLAP does not require `resources` to +> be set for autoscaling to work. + +### Helm Values Reference (Autoscaling) + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster..replicas` | `1-2` | Static replica count, or max replicas ceiling when autoscaling is enabled | +| `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling | +| `cluster..autoscaling.minReplicas` | `0` (HS2/LLAP/TezAM), `1` (HMS) | Minimum replica count. Set to 0 for scale-to-zero | +| `cluster..autoscaling.scaleUpThreshold` | varies | Metric threshold triggering scale-up | +| `cluster..autoscaling.scaleDownThreshold` | varies | Metric threshold triggering scale-down | +| `cluster..autoscaling.cooldownSeconds` | varies | Cooldown after a scaling event | +| `cluster..autoscaling.gracePeriodSeconds` | varies | Max drain time before forced termination | + +--- + ## Connect to HiveServer2 +HiveServer2 runs in **HTTP transport mode** by default (recommended for Kubernetes +environments as it works well with load balancers, ingress controllers, and proxies). + +### Standard Connection (minReplicas >= 1) + +When HS2 always has at least one pod running, connect directly to the service: + ```bash -kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hiveserver2:10000/" +kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hiveserver2:10001/;transportMode=http;httpPath=cliservice" ``` Or via port-forward: ```bash -kubectl port-forward svc/hive-hiveserver2 10000:10000 -beeline -u "jdbc:hive2://localhost:10000/" +kubectl port-forward svc/hive-hiveserver2 10001:10001 +beeline -u "jdbc:hive2://localhost:10001/;transportMode=http;httpPath=cliservice" +``` + +### Connecting with Scale-to-Zero (minReplicas = 0) + +When HS2 is configured with `minReplicas: 0`, the deployment starts with zero pods. +Connections go through the **KEDA HTTP interceptor proxy** which automatically wakes +HS2 when a request arrives (first request takes ~30-60s while the pod starts). + +``` +Traffic flow: +Client → KEDA HTTP Interceptor → (if 0 pods: scale up, wait) → HS2 Service → HS2 Pod +``` + +**Via port-forward (local development):** + +```bash +# Port-forward the KEDA HTTP interceptor proxy +kubectl port-forward -n keda svc/keda-add-ons-http-interceptor-proxy 8080:8080 + +# Connect — interceptor auto-wakes HS2 (first request may take 30-60s) +beeline -u "jdbc:hive2://localhost:8080/;transportMode=http;httpPath=cliservice" +``` + +**Via Ingress (production):** + +Create an Ingress that routes your domain to the KEDA interceptor. The key is the +`upstream-vhost` annotation which rewrites the Host header to the internal service +name so the interceptor can match it — no extra operator configuration needed: + +```bash +cat <<'EOF' | kubectl apply -f - +apiVersion: networking.k8s.io/v1 +kind: Ingress +metadata: + name: hive-interceptor + namespace: keda + annotations: + # Rewrite Host header to internal service name so KEDA interceptor can route it + nginx.ingress.kubernetes.io/upstream-vhost: "hive-hiveserver2.default.svc.cluster.local" +spec: + ingressClassName: nginx + rules: + - host: hive.example.com + http: + paths: + - path: / + pathType: Prefix + backend: + service: + name: keda-add-ons-http-interceptor-proxy + port: + number: 8080 +EOF +``` + +Connect via beeline using the Ingress: + +```bash +beeline -u "jdbc:hive2://hive.example.com:80/;transportMode=http;httpPath=cliservice" +``` + +**Manual wake (fallback without HTTP Add-on):** + +```bash +kubectl scale deployment/hive-hiveserver2 --replicas=1 +kubectl wait --for=condition=ready pod -l app.kubernetes.io/component=hiveserver2 --timeout=120s +kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hiveserver2:10001/;transportMode=http;httpPath=cliservice" ``` +> **Note:** The operator sets `hive.server2.transport.mode=http`, +> `hive.server2.thrift.http.port=10001`, and +> `hive.server2.thrift.http.path=cliservice` by default. The binary Thrift +> port (10000) is still exposed for backward compatibility but HTTP mode +> is the primary transport. To override, use `configOverrides` in the +> HiveServer2 spec. + --- ## Helm Values Reference @@ -620,6 +1050,18 @@ beeline -u "jdbc:hive2://localhost:10000/" | `cluster.tezAm.extraVolumes` | `[]` | Additional volumes for TezAM pods | | `cluster.tezAm.extraVolumeMounts` | `[]` | Additional volume mounts for TezAM containers | +### Autoscaling (per component) + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling for this component | +| `cluster..autoscaling.minReplicas` | `2` | Floor replica count during scale-down | +| `cluster..autoscaling.scaleUpThreshold` | `60-80` | Metric threshold triggering scale-up (CPU% for HS2/HMS/TezAM with resources; pending tasks per AM for TezAM without resources; queue depth for LLAP) | +| `cluster..autoscaling.scaleDownThreshold` | `10-30` | Metric percentage threshold triggering scale-down | +| `cluster..autoscaling.cooldownSeconds` | `300-900` | Minimum seconds between scaling events | +| `cluster..autoscaling.gracePeriodSeconds` | `60-600` | Max time (seconds) to wait for graceful drain | +| `cluster.hiveServer2.autoscaling.scaleToZeroHosts` | `[]` | Hostnames for KEDA HTTP interceptor routing (Ingress domain) | + --- ## Upgrade and Uninstall @@ -659,12 +1101,35 @@ helm install hive ./helm/hive-operator -f my-values.yaml ### Remove Everything (including dependencies) ```bash +# 1. Uninstall Hive operator (removes ScaledObjects, pods, services via owner references) helm uninstall hive -kubectl delete crd hiveclusters.hive.apache.org +kubectl delete crd hiveclusters.hive.apache.org --ignore-not-found + +# 2. Remove HS2 Ingress (if configured for scale-to-zero wake) +kubectl delete ingress hive-hs2-ingress --ignore-not-found + +# 3. Uninstall autoscaling infrastructure (KEDA, HTTP Add-on, Prometheus) +helm uninstall http-add-on -n keda --ignore-not-found +helm uninstall keda -n keda --ignore-not-found +helm uninstall prometheus -n monitoring --ignore-not-found + +# 4. Remove KEDA CRDs (not removed by helm uninstall) +kubectl delete crd --ignore-not-found \ + scaledobjects.keda.sh \ + scaledjobs.keda.sh \ + triggerauthentications.keda.sh \ + clustertriggerauthentications.keda.sh \ + httpscaledobjects.http.keda.sh + +# 5. Uninstall storage and infrastructure dependencies helm uninstall ozone postgres zookeeper --ignore-not-found + +# 6. Clean up PVCs, secrets, and namespaces kubectl delete pvc data-zookeeper-0 --ignore-not-found kubectl delete pvc data-postgres-postgresql-0 --ignore-not-found kubectl delete secret hive-db-secret --ignore-not-found +kubectl delete namespace keda --ignore-not-found +kubectl delete namespace monitoring --ignore-not-found ``` --- diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 99768633a128..6fc5916b84dd 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -44,6 +44,44 @@ spec: hiveServer2: description: HiveServer2 component configuration properties: + autoscaling: + description: Autoscaling configuration (requires KEDA installed + in the cluster) + properties: + cooldownSeconds: + default: 600 + description: Cooldown period in seconds after a scaling event + before another can occur + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 300 + description: Maximum time in seconds to wait for graceful + drain during scale-down before the pod is forcibly terminated + type: integer + minReplicas: + default: 0 + description: Minimum number of replicas (floor for scale-down). + Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on + for wake-from-zero) + type: integer + scaleDownThreshold: + default: 20 + description: "Percentage threshold that triggers scale-down\ + \ (all conditions must be met: metric below threshold AND\ + \ CPU below threshold)" + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ + \ with resources, pending tasks per AM for TezAM without\ + \ resources)" + type: integer + type: object configOverrides: additionalProperties: type: string @@ -152,6 +190,44 @@ spec: llap: description: LLAP daemon configuration. Enabled by default. properties: + autoscaling: + description: Autoscaling configuration (requires KEDA installed + in the cluster) + properties: + cooldownSeconds: + default: 600 + description: Cooldown period in seconds after a scaling event + before another can occur + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 300 + description: Maximum time in seconds to wait for graceful + drain during scale-down before the pod is forcibly terminated + type: integer + minReplicas: + default: 0 + description: Minimum number of replicas (floor for scale-down). + Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on + for wake-from-zero) + type: integer + scaleDownThreshold: + default: 20 + description: "Percentage threshold that triggers scale-down\ + \ (all conditions must be met: metric below threshold AND\ + \ CPU below threshold)" + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ + \ with resources, pending tasks per AM for TezAM without\ + \ resources)" + type: integer + type: object configOverrides: additionalProperties: type: string @@ -235,6 +311,44 @@ spec: metastore: description: Metastore component configuration properties: + autoscaling: + description: Autoscaling configuration (requires KEDA installed + in the cluster) + properties: + cooldownSeconds: + default: 600 + description: Cooldown period in seconds after a scaling event + before another can occur + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 300 + description: Maximum time in seconds to wait for graceful + drain during scale-down before the pod is forcibly terminated + type: integer + minReplicas: + default: 0 + description: Minimum number of replicas (floor for scale-down). + Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on + for wake-from-zero) + type: integer + scaleDownThreshold: + default: 20 + description: "Percentage threshold that triggers scale-down\ + \ (all conditions must be met: metric below threshold AND\ + \ CPU below threshold)" + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ + \ with resources, pending tasks per AM for TezAM without\ + \ resources)" + type: integer + type: object configOverrides: additionalProperties: type: string @@ -371,6 +485,44 @@ spec: tezAm: description: Tez Application Master configuration. Enabled by default. properties: + autoscaling: + description: Autoscaling configuration (requires KEDA installed + in the cluster) + properties: + cooldownSeconds: + default: 600 + description: Cooldown period in seconds after a scaling event + before another can occur + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 300 + description: Maximum time in seconds to wait for graceful + drain during scale-down before the pod is forcibly terminated + type: integer + minReplicas: + default: 0 + description: Minimum number of replicas (floor for scale-down). + Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on + for wake-from-zero) + type: integer + scaleDownThreshold: + default: 20 + description: "Percentage threshold that triggers scale-down\ + \ (all conditions must be met: metric below threshold AND\ + \ CPU below threshold)" + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ + \ with resources, pending tasks per AM for TezAM without\ + \ resources)" + type: integer + type: object configOverrides: additionalProperties: type: string diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml index d27e1fea8c6f..791c60e0d813 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml @@ -50,3 +50,15 @@ rules: - apiGroups: [""] resources: ["pods"] verbs: ["get", "list", "watch"] + # PodDisruptionBudgets for graceful autoscaling + - apiGroups: ["policy"] + resources: ["poddisruptionbudgets"] + verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] + # KEDA ScaledObjects for autoscaling + - apiGroups: ["keda.sh"] + resources: ["scaledobjects", "triggerauthentications"] + verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] + # KEDA HTTP Add-on for scale-to-zero (wake-from-zero on HTTP request) + - apiGroups: ["http.keda.sh"] + resources: ["httpscaledobjects"] + verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index 091ecefb3cb0..c490b1f5d333 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -67,6 +67,15 @@ spec: extraVolumeMounts: {{- toYaml .Values.cluster.metastore.extraVolumeMounts | nindent 6 }} {{- end }} + {{- if and .Values.cluster.metastore.autoscaling .Values.cluster.metastore.autoscaling.enabled }} + autoscaling: + enabled: true + minReplicas: {{ .Values.cluster.metastore.autoscaling.minReplicas }} + scaleUpThreshold: {{ .Values.cluster.metastore.autoscaling.scaleUpThreshold }} + scaleDownThreshold: {{ .Values.cluster.metastore.autoscaling.scaleDownThreshold }} + cooldownSeconds: {{ .Values.cluster.metastore.autoscaling.cooldownSeconds }} + gracePeriodSeconds: {{ .Values.cluster.metastore.autoscaling.gracePeriodSeconds }} + {{- end }} {{- else }} {{- if .Values.cluster.metastore.externalUri }} externalUri: {{ .Values.cluster.metastore.externalUri | quote }} @@ -96,6 +105,15 @@ spec: extraVolumeMounts: {{- toYaml .Values.cluster.hiveServer2.extraVolumeMounts | nindent 6 }} {{- end }} + {{- if and .Values.cluster.hiveServer2.autoscaling .Values.cluster.hiveServer2.autoscaling.enabled }} + autoscaling: + enabled: true + minReplicas: {{ .Values.cluster.hiveServer2.autoscaling.minReplicas }} + scaleUpThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleUpThreshold }} + scaleDownThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownThreshold }} + cooldownSeconds: {{ .Values.cluster.hiveServer2.autoscaling.cooldownSeconds }} + gracePeriodSeconds: {{ .Values.cluster.hiveServer2.autoscaling.gracePeriodSeconds }} + {{- end }} llap: enabled: {{ .Values.cluster.llap.enabled }} @@ -120,6 +138,15 @@ spec: extraVolumeMounts: {{- toYaml .Values.cluster.llap.extraVolumeMounts | nindent 6 }} {{- end }} + {{- if and .Values.cluster.llap.autoscaling .Values.cluster.llap.autoscaling.enabled }} + autoscaling: + enabled: true + minReplicas: {{ .Values.cluster.llap.autoscaling.minReplicas }} + scaleUpThreshold: {{ .Values.cluster.llap.autoscaling.scaleUpThreshold }} + scaleDownThreshold: {{ .Values.cluster.llap.autoscaling.scaleDownThreshold }} + cooldownSeconds: {{ .Values.cluster.llap.autoscaling.cooldownSeconds }} + gracePeriodSeconds: {{ .Values.cluster.llap.autoscaling.gracePeriodSeconds }} + {{- end }} {{- end }} tezAm: @@ -146,6 +173,15 @@ spec: extraVolumeMounts: {{- toYaml .Values.cluster.tezAm.extraVolumeMounts | nindent 6 }} {{- end }} + {{- if and .Values.cluster.tezAm.autoscaling .Values.cluster.tezAm.autoscaling.enabled }} + autoscaling: + enabled: true + minReplicas: {{ .Values.cluster.tezAm.autoscaling.minReplicas }} + scaleUpThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleUpThreshold }} + scaleDownThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleDownThreshold }} + cooldownSeconds: {{ .Values.cluster.tezAm.autoscaling.cooldownSeconds }} + gracePeriodSeconds: {{ .Values.cluster.tezAm.autoscaling.gracePeriodSeconds }} + {{- end }} {{- end }} zookeeper: diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index b7d75930c5b2..a0823f90de0d 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -112,6 +112,15 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # Autoscaling (requires KEDA + Prometheus in the cluster) + # When enabled, 'replicas' above acts as the max replica ceiling + autoscaling: + enabled: false + minReplicas: 1 + scaleUpThreshold: 75 + scaleDownThreshold: 30 + cooldownSeconds: 300 + gracePeriodSeconds: 60 # Set to use an external Metastore instead of deploying one: # enabled: false # externalUri: "thrift://external-metastore:9083" @@ -127,6 +136,16 @@ cluster: externalJars: [] extraVolumes: [] extraVolumeMounts: [] + # Autoscaling (requires KEDA + Prometheus + KEDA HTTP Add-on in the cluster) + # minReplicas: 0 enables scale-to-zero — beeline HTTP connects wake HS2 via KEDA HTTP interceptor + # When enabled, 'replicas' above acts as the max replica ceiling + autoscaling: + enabled: false + minReplicas: 0 + scaleUpThreshold: 80 + scaleDownThreshold: 20 + cooldownSeconds: 600 + gracePeriodSeconds: 300 # --------------------------------------------------------------------------- # LLAP — enabled by default for full-HA @@ -141,6 +160,16 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # Autoscaling (requires KEDA + Prometheus in the cluster) + # minReplicas: 0 enables scale-to-zero — scales up immediately when queries need LLAP + # When enabled, 'replicas' above acts as the max replica ceiling + autoscaling: + enabled: false + minReplicas: 0 + scaleUpThreshold: 1 + scaleDownThreshold: 0 + cooldownSeconds: 900 + gracePeriodSeconds: 600 # --------------------------------------------------------------------------- # TEZ AM — enabled by default for full-HA @@ -154,3 +183,15 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # Autoscaling (requires KEDA + Prometheus in the cluster) + # minReplicas: 0 enables scale-to-zero — wakes when HS2 receives queries + # When enabled, 'replicas' above acts as the max replica ceiling + # scaleUpThreshold: with CPU resources set → CPU% (e.g., 60 = 60% utilization); + # without CPU resources → pending tasks per AM (e.g., 5 = scale when 5+ tasks waiting) + autoscaling: + enabled: false + minReplicas: 0 + scaleUpThreshold: 5 + scaleDownThreshold: 10 + cooldownSeconds: 600 + gracePeriodSeconds: 120 diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index cc2eb0de6de0..9b1cb75d6553 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -45,11 +45,13 @@ import io.javaoperatorsdk.operator.processing.dependent.Matcher; import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.DatabaseConfig; import org.apache.hive.kubernetes.operator.model.spec.ResourceRequirementsSpec; import org.apache.hive.kubernetes.operator.model.spec.SecretKeyRef; import org.apache.hive.kubernetes.operator.model.spec.ProbeSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -422,4 +424,141 @@ protected static Probe buildTcpProbe(int port, ProbeSpec spec, int defaultInitia return builder.build(); } + /** Path where the JMX Exporter agent JAR is stored inside the pod. */ + protected static final String JMX_EXPORTER_DIR = "/opt/jmx-exporter"; + protected static final String JMX_EXPORTER_JAR = JMX_EXPORTER_DIR + "/jmx_prometheus_javaagent.jar"; + protected static final String JMX_EXPORTER_CONFIG = JMX_EXPORTER_DIR + "/config.yaml"; + + /** + * Adds the Prometheus JMX Exporter agent infrastructure to a pod spec when + * autoscaling is enabled. This includes: + *
    + *
  • An emptyDir volume for the JMX exporter JAR and config
  • + *
  • An init container that downloads the agent JAR and writes a config file
  • + *
  • A volume mount on the main container
  • + *
  • A container port for the metrics endpoint (9404)
  • + *
  • The javaagent JVM argument appended to SERVICE_OPTS
  • + *
+ * + * @param image the container image (used for the init container) + * @param component the Hive component name (for JMX bean pattern matching) + * @param initContainers list to add the download init container to + * @param volumeMounts list to add the jmx-exporter mount to (main container) + * @param volumes list to add the emptyDir volume to + * @param envVars list of env vars — SERVICE_OPTS will be updated with the javaagent flag + * @param ports list to add the metrics port to + */ + protected static void addJmxExporter( + String image, String component, + List initContainers, + List volumeMounts, + List volumes, + List envVars, + List ports) { + + // Volume for the JMX exporter JAR + config + volumes.add(new VolumeBuilder() + .withName("jmx-exporter") + .withNewEmptyDir().endEmptyDir().build()); + VolumeMount exporterMount = new VolumeMountBuilder() + .withName("jmx-exporter") + .withMountPath(JMX_EXPORTER_DIR).build(); + volumeMounts.add(exporterMount); + + // JMX exporter config: export all beans in a catch-all pattern + // The agent exposes metrics in Prometheus text format at /metrics + String jmxConfig = buildJmxExporterConfig(component); + + // Init container: download JAR + write config + String downloadCmd = String.format( + "wget -q --tries=3 --waitretry=5 -O %s '%s' && " + + "cat > %s << 'JMXEOF'\n%s\nJMXEOF", + JMX_EXPORTER_JAR, ConfigUtils.JMX_EXPORTER_JAR_URL, + JMX_EXPORTER_CONFIG, jmxConfig); + initContainers.add(new ContainerBuilder() + .withName("jmx-exporter-init") + .withImage(image) + .withCommand("/bin/bash", "-c", downloadCmd) + .withVolumeMounts(exporterMount) + .build()); + + // Expose the metrics port + ports.add(new io.fabric8.kubernetes.api.model.ContainerPortBuilder() + .withName("metrics") + .withContainerPort(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT).build()); + + // Add javaagent flag to the appropriate JVM opts env var. + // LLAP uses LLAP_DAEMON_OPTS (its startup script ignores SERVICE_OPTS). + String agentArg = String.format("-javaagent:%s=%d:%s", + JMX_EXPORTER_JAR, ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT, JMX_EXPORTER_CONFIG); + String optsEnvVar = "llap".equals(component) ? "LLAP_DAEMON_OPTS" : "SERVICE_OPTS"; + boolean found = false; + for (int i = 0; i < envVars.size(); i++) { + if (optsEnvVar.equals(envVars.get(i).getName())) { + String existing = envVars.get(i).getValue(); + envVars.set(i, new EnvVar(optsEnvVar, + existing + " " + agentArg, null)); + found = true; + break; + } + } + if (!found) { + envVars.add(new EnvVar(optsEnvVar, agentArg, null)); + } + } + + /** + * Builds the JMX Exporter YAML config for a Hive component. + * Uses broad patterns to export all Hive/Hadoop metrics relevant to autoscaling. + */ + private static String buildJmxExporterConfig(String component) { + StringBuilder sb = new StringBuilder(); + sb.append("lowercaseOutputName: true\n"); + sb.append("lowercaseOutputLabelNames: true\n"); + sb.append("rules:\n"); + + switch (component) { + case "hiveserver2": + // HS2 session and operation metrics + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: hs2_$1\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: hs2_active_calls_$1\n"); + sb.append(" type: GAUGE\n"); + // Tez session pool metrics (pending tasks, backlog ratio, running tasks) + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: tez_session_$1\n"); + sb.append(" type: GAUGE\n"); + break; + case "metastore": + // HMS API call metrics + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: api_$1_total\n"); + sb.append(" type: COUNTER\n"); + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: hive_metastore_open_connections\n"); + sb.append(" type: GAUGE\n"); + break; + case "llap": + // LLAP uses its own MetricsSystem (not DefaultMetricsSystem). + // Default JMX exporter pattern (.*) exports Hadoop Metrics2 MBeans as: + // hadoop_llapdaemon_{name=""} + // e.g., hadoop_llapdaemon_executornumqueuedrequests{name="LlapDaemonExecutorMetrics-..."} + // No custom rules needed — the default naming is usable directly. + sb.append("- pattern: '.*'\n"); + break; + case "tezam": + // TezAM DAG execution metrics + sb.append("- pattern: 'Hadoop<>(.+)'\n"); + sb.append(" name: tez_am_$1\n"); + sb.append(" type: GAUGE\n"); + break; + default: + sb.append("- pattern: '.*'\n"); + break; + } + return sb.toString(); + } + } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java new file mode 100644 index 000000000000..feff8775a6f4 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java @@ -0,0 +1,73 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.Optional; +import java.util.Set; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.GarbageCollected; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import io.javaoperatorsdk.operator.processing.dependent.Creator; +import io.javaoperatorsdk.operator.processing.dependent.Updater; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.GenericKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Base class for dependent resources that manage custom resources via + * {@link GenericKubernetesResource} (e.g. KEDA ScaledObject, HTTPScaledObject). + *

+ * Extends {@link GenericKubernetesDependentResource} which properly configures + * the informer with the specified GroupVersionKind, avoiding the fabric8 + * "resources cannot be called with a generic type" error. + *

+ * Also overrides {@link #getSecondaryResource} to use the dependent's own + * event source (same pattern as {@link HiveDependentResource}) so multiple + * GenericKubernetesResource dependents don't collide in the type-based lookup. + */ +public abstract class HiveGenericDependentResource + extends GenericKubernetesDependentResource + implements Creator, + Updater, + GarbageCollected { + + protected HiveGenericDependentResource(GroupVersionKind gvk) { + super(gvk); + } + + /** + * Returns the expected Kubernetes resource name for this dependent given the primary. + * Used to discriminate between multiple secondary resources of the same GVK + * (e.g. multiple ScaledObjects owned by the same HiveCluster). + */ + protected abstract String getResourceName(HiveCluster hiveCluster); + + @Override + public Optional getSecondaryResource( + HiveCluster primary, Context context) { + String expectedName = getResourceName(primary); + Set secondaries = eventSource() + .map(es -> es.getSecondaryResources(primary)) + .orElse(Set.of()); + return secondaries.stream() + .filter(r -> expectedName.equals(r.getMetadata().getName())) + .findFirst(); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index ccb3048dea98..c61383ac0f5a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -26,6 +26,8 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.Lifecycle; +import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder; @@ -34,6 +36,7 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.HiveServer2Spec; import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; @@ -125,21 +128,28 @@ protected Deployment desired(HiveCluster hiveCluster, hs2.configOverrides(), ConfigUtils.HIVE_SERVER2_THRIFT_PORT_KEY, null, ConfigUtils.HIVE_SERVER2_THRIFT_PORT_DEFAULT); + int hs2HttpPort = ConfigUtils.getInt( + hs2.configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); int hs2WebUiPort = ConfigUtils.getInt( hs2.configOverrides(), ConfigUtils.HIVE_SERVER2_WEBUI_PORT_KEY, null, ConfigUtils.HIVE_SERVER2_WEBUI_PORT_DEFAULT); - List ports = List.of( - new ContainerPortBuilder() - .withName("thrift") - .withContainerPort(hs2ThriftPort).build(), - new ContainerPortBuilder() - .withName("webui") - .withContainerPort(hs2WebUiPort).build() - ); + List ports = new ArrayList<>(); + ports.add(new ContainerPortBuilder() + .withName("thrift") + .withContainerPort(hs2ThriftPort).build()); + ports.add(new ContainerPortBuilder() + .withName("http") + .withContainerPort(hs2HttpPort).build()); + ports.add(new ContainerPortBuilder() + .withName("webui") + .withContainerPort(hs2WebUiPort).build()); - Probe readinessProbe = buildTcpProbe(hs2ThriftPort, hs2.readinessProbe(), 15, 10, 3); - Probe livenessProbe = buildTcpProbe(hs2ThriftPort, hs2.livenessProbe(), 120, 30, 10); + // Probes target the HTTP transport port (default mode) + Probe readinessProbe = buildTcpProbe(hs2HttpPort, hs2.readinessProbe(), 15, 10, 3); + Probe livenessProbe = buildTcpProbe(hs2HttpPort, hs2.livenessProbe(), 120, 30, 10); boolean tezAmEnabled = spec.tezAm().isEnabled(); @@ -185,6 +195,13 @@ protected Deployment desired(HiveCluster hiveCluster, replaceConfMountWithSubPaths(volumeMounts, "hive-config", "hive-site.xml", "tez-site.xml", "core-site.xml"); + // Add Prometheus JMX Exporter when autoscaling is enabled + AutoscalingSpec autoscaling = hs2.autoscaling(); + if (autoscaling.isEnabled()) { + addJmxExporter(spec.image(), COMPONENT, + initContainers, volumeMounts, volumes, envVars, ports); + } + // Pre-compute config hash for the pod template annotation. // This ensures the Deployment is created with the correct hash // from the start (single ReplicaSet) and triggers rolling @@ -194,6 +211,19 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + // When autoscaling is enabled and the Deployment already exists, preserve the current + // replica count (managed by KEDA/HPA). On initial creation: + // - minReplicas == 0 (scale-to-zero): start at 0, KEDA HTTPScaledObject handles wake-up + // - minReplicas > 0: start at configured replicas + boolean autoscalingEnabled = hs2.autoscaling() != null && hs2.autoscaling().isEnabled(); + Integer replicas = hs2.replicas(); + if (autoscalingEnabled) { + int initialReplicas = hs2.autoscaling().minReplicas() == 0 ? 0 : hs2.replicas(); + replicas = getSecondaryResource(hiveCluster, context) + .map(d -> d.getSpec().getReplicas()) + .orElse(initialReplicas); + } + Deployment deployment = new DeploymentBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -201,7 +231,7 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(hs2.replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() @@ -233,6 +263,58 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); + // Graceful scale-down: deregister from ZK, then poll JMX Exporter (port 9404) for sessions. + // Uses flat Prometheus text format — same metric KEDA reads — not brittle JSON parsing. + if (autoscaling.isEnabled()) { + String preStopScript = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Deregistering HiveServer2 from ZooKeeper...'", + "hive --service hiveserver2 --deregister || echo '[preStop] WARNING: ZK deregister failed'", + "echo '[preStop] Waiting for open sessions to drain (polling localhost:9404/metrics)...'", + "RETRIES=0", + "while true; do", + " RESPONSE=$(curl -sf http://localhost:9404/metrics)", + " if [ $? -ne 0 ]; then", + " RETRIES=$((RETRIES+1))", + " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", + " if [ $RETRIES -ge 6 ]; then", + " echo '[preStop] JMX Exporter not responding after 30s. Proceeding with shutdown.'", + " break", + " fi", + " sleep 5; continue", + " fi", + " SESSIONS=$(echo \"$RESPONSE\" | grep '^hs2_open_sessions ' | awk '{print $2}')", + " if [ -z \"$SESSIONS\" ]; then", + " echo '[preStop] WARNING: hs2_open_sessions metric not found. JMX Exporter may not be configured.'", + " break", + " fi", + " if [ \"${SESSIONS%.*}\" -le 0 ] 2>/dev/null; then", + " echo '[preStop] All sessions drained. Shutting down.'", + " break", + " fi", + " echo \"[preStop] hs2_open_sessions=$SESSIONS — waiting...\"", + " RETRIES=0", + " sleep 5", + "done"); + Lifecycle lifecycle = new LifecycleBuilder() + .withNewPreStop() + .withNewExec() + .withCommand("/bin/bash", "-c", preStopScript) + .endExec() + .endPreStop() + .build(); + deployment.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); + deployment.getSpec().getTemplate().getSpec() + .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); + // Prometheus scrape annotations for JMX Exporter metrics endpoint + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/scrape", "true"); + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/path", "/metrics"); + } + if (spec.volumes() != null) { deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java new file mode 100644 index 000000000000..74794ad8e16a --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java @@ -0,0 +1,129 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA HTTPScaledObject for HiveServer2 scale-to-zero. + *

+ * Requires the KEDA HTTP Add-on to be installed in the cluster. + * The HTTP Add-on creates an interceptor proxy that: + *

    + *
  • Sits in front of the HS2 Service
  • + *
  • Queues incoming beeline/HTTP requests when HS2 has 0 pods
  • + *
  • Triggers KEDA to scale HS2 from 0 to 1
  • + *
  • Forwards the queued request once a pod is ready
  • + *
+ *

+ * This dependent is activated ONLY when minReplicas == 0 (scale-to-zero mode). + * When minReplicas > 0, the regular ScaledObject (Prometheus-based) is used instead. + */ +public class HiveServer2HttpScaledObjectDependent extends HiveGenericDependentResource { + + public HiveServer2HttpScaledObjectDependent() { + super(new GroupVersionKind("http.keda.sh", "v1alpha1", "HTTPScaledObject")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); + int maxReplicas = hiveCluster.getSpec().hiveServer2().replicas(); + String clusterName = hiveCluster.getMetadata().getName(); + String namespace = hiveCluster.getMetadata().getNamespace(); + String deploymentName = clusterName + "-hiveserver2"; + String serviceName = clusterName + "-hiveserver2"; + + int httpPort = ConfigUtils.getInt( + hiveCluster.getSpec().hiveServer2().configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); + + Map spec = new HashMap<>(); + + // Hosts the interceptor matches for routing. + // Uses internal service DNS names (Ingress rewrites Host header to match these) + // plus localhost for kubectl port-forward scenarios. + spec.put("hosts", List.of( + serviceName + "." + namespace + ".svc.cluster.local", + serviceName, + "localhost" + )); + spec.put("pathPrefixes", List.of("/")); + + // Target deployment and service + spec.put("scaleTargetRef", Map.of( + "name", deploymentName, + "kind", "Deployment", + "apiVersion", "apps/v1", + "service", serviceName, + "port", httpPort + )); + + // Replica bounds + spec.put("replicas", Map.of( + "min", 0, + "max", maxReplicas + )); + + // Scaling metric: scale up when there are pending requests + spec.put("scalingMetric", Map.of( + "requestRate", Map.of( + "granularity", "1s", + "targetValue", autoscaling.scaleUpThreshold(), + "window", "1m" + ) + )); + + // Cooldown before scaling back to 0 + spec.put("scaledownPeriod", autoscaling.cooldownSeconds()); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("http.keda.sh/v1alpha1") + .withKind("HTTPScaledObject") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(namespace) + .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hiveserver2-httpso"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java new file mode 100644 index 000000000000..054881f9644d --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java @@ -0,0 +1,62 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * PodDisruptionBudget for HiveServer2. + * Ensures at least one HS2 pod remains available during voluntary disruptions + * (scale-down, node drain, rolling updates) to prevent query failures. + */ +public class HiveServer2PdbDependent + extends CRUDKubernetesDependentResource { + + public HiveServer2PdbDependent() { + super(PodDisruptionBudget.class); + } + + @Override + protected PodDisruptionBudget desired(HiveCluster hiveCluster, + Context context) { + return new PodDisruptionBudgetBuilder() + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) + .endMetadata() + .withNewSpec() + .withMinAvailable(new IntOrString(1)) + .withNewSelector() + .withMatchLabels(Labels.selectorForComponent(hiveCluster, "hiveserver2")) + .endSelector() + .endSpec() + .build(); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hiveserver2-pdb"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java new file mode 100644 index 000000000000..196d97ce8b8f --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java @@ -0,0 +1,149 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA ScaledObject for HiveServer2 autoscaling. + *

+ * Scale-up triggers (OR): + * - hs2_active_sessions > scaleUpThreshold% of hive.server2.session.max (1 min) + * - Pod CPU > 75% + *

+ * Scale-down triggers (AND): + * - hs2_open_sessions < scaleDownThreshold% of max + * - CPU < 30% + *

+ * Cooldown: configurable (default 600s / 10 minutes) + */ +public class HiveServer2ScaledObjectDependent extends HiveGenericDependentResource { + + public HiveServer2ScaledObjectDependent() { + super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); + int maxReplicas = hiveCluster.getSpec().hiveServer2().replicas(); + String targetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; + + Map spec = new HashMap<>(); + spec.put("scaleTargetRef", Map.of( + "apiVersion", "apps/v1", + "kind", "Deployment", + "name", targetName + )); + // KEDA requires idleReplicaCount < minReplicaCount. + // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). + int minReplicaCount = Math.max(1, autoscaling.minReplicas()); + spec.put("minReplicaCount", minReplicaCount); + spec.put("maxReplicaCount", maxReplicas); + if (autoscaling.minReplicas() == 0) { + spec.put("idleReplicaCount", 0); + } + spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); + spec.put("pollingInterval", 30); + + // Advanced scaling policy: scale down one pod at a time for graceful drain + spec.put("advanced", Map.of( + "horizontalPodAutoscalerConfig", Map.of( + "behavior", Map.of( + "scaleDown", Map.of( + "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), + "policies", List.of(Map.of( + "type", "Pods", + "value", 1, + "periodSeconds", 60 + )) + ), + "scaleUp", Map.of( + "stabilizationWindowSeconds", 60, + "policies", List.of(Map.of( + "type", "Percent", + "value", 100, + "periodSeconds", 60 + )) + ) + ) + ) + )); + + // Triggers: Prometheus for hs2_active_sessions + CPU fallback (only when CPU requests defined) + // "or vector(0)" ensures the query returns 0 (not empty) when HS2 has no pods. + List> triggers = new ArrayList<>(); + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_active_sessions", + "query", String.format( + "avg(hs2_active_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + )); + if (hiveCluster.getSpec().hiveServer2().resources() != null) { + // activationValue prevents idle JVM CPU from keeping the ScaledObject active. + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", "75", + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + } + spec.put("triggers", triggers); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("keda.sh/v1alpha1") + .withKind("ScaledObject") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hiveserver2-scaledobject"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java index a9707ac0dfa6..13b218986e67 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java @@ -48,6 +48,9 @@ protected Service desired(HiveCluster hiveCluster, int thriftPort = ConfigUtils.getInt(hs2.configOverrides(), ConfigUtils.HIVE_SERVER2_THRIFT_PORT_KEY, null, ConfigUtils.HIVE_SERVER2_THRIFT_PORT_DEFAULT); + int httpPort = ConfigUtils.getInt(hs2.configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); int webUiPort = ConfigUtils.getInt(hs2.configOverrides(), ConfigUtils.HIVE_SERVER2_WEBUI_PORT_KEY, null, ConfigUtils.HIVE_SERVER2_WEBUI_PORT_DEFAULT); @@ -68,6 +71,11 @@ protected Service desired(HiveCluster hiveCluster, .withPort(thriftPort) .withTargetPort(new IntOrString(thriftPort)) .endPort() + .addNewPort() + .withName("http") + .withPort(httpPort) + .withTargetPort(new IntOrString(httpPort)) + .endPort() .addNewPort() .withName("webui") .withPort(webUiPort) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java new file mode 100644 index 000000000000..1f077751aa61 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java @@ -0,0 +1,62 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * PodDisruptionBudget for LLAP daemons. + * Ensures at least one LLAP daemon remains available during voluntary disruptions + * to prevent query failures and cache loss. + */ +public class LlapPdbDependent + extends CRUDKubernetesDependentResource { + + public LlapPdbDependent() { + super(PodDisruptionBudget.class); + } + + @Override + protected PodDisruptionBudget desired(HiveCluster hiveCluster, + Context context) { + return new PodDisruptionBudgetBuilder() + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "llap")) + .endMetadata() + .withNewSpec() + .withMinAvailable(new IntOrString(1)) + .withNewSelector() + .withMatchLabels(Labels.selectorForComponent(hiveCluster, "llap")) + .endSelector() + .endSpec() + .build(); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-llap-pdb"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java new file mode 100644 index 000000000000..7f6886a594df --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java @@ -0,0 +1,158 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA ScaledObject for LLAP daemon autoscaling. + *

+ * Scale-up trigger: + * - NumQueuedRequests > 0 for 1 minute (queue non-empty means all executors are busy) + *

+ * Scale-down trigger: + * - NumExecutorsAvailable == NumExecutors (daemon completely idle) + *

+ * Cooldown: configurable (default 900s / 15 minutes — scaling down destroys in-memory cache) + */ +public class LlapScaledObjectDependent extends HiveGenericDependentResource { + + public LlapScaledObjectDependent() { + super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().llap().autoscaling(); + int maxReplicas = hiveCluster.getSpec().llap().replicas(); + String targetName = hiveCluster.getMetadata().getName() + "-llap"; + + Map spec = new HashMap<>(); + spec.put("scaleTargetRef", Map.of( + "apiVersion", "apps/v1", + "kind", "StatefulSet", + "name", targetName + )); + // KEDA requires idleReplicaCount < minReplicaCount. + // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). + int minReplicaCount = Math.max(1, autoscaling.minReplicas()); + spec.put("minReplicaCount", minReplicaCount); + spec.put("maxReplicaCount", maxReplicas); + if (autoscaling.minReplicas() == 0) { + spec.put("idleReplicaCount", 0); + } + spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); + spec.put("pollingInterval", 5); + + // LLAP scale-up is aggressive: when queries need daemons, scale immediately to max. + // Scale down is slow (1 pod per cooldown) to preserve in-memory cache. + spec.put("advanced", Map.of( + "horizontalPodAutoscalerConfig", Map.of( + "behavior", Map.of( + "scaleDown", Map.of( + "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), + "policies", List.of(Map.of( + "type", "Pods", + "value", 1, + "periodSeconds", autoscaling.cooldownSeconds() + )) + ), + "scaleUp", Map.of( + "stabilizationWindowSeconds", 0, + "policies", List.of(Map.of( + "type", "Pods", + "value", maxReplicas, + "periodSeconds", 15 + )) + ) + ) + ) + )); + + // Triggers: + // 1. Prometheus for NumQueuedRequests — drives proportional scaling. + // More queued requests = more LLAP daemons needed. Scales up to max. + // 2. HS2 open sessions — activation only (wake from 0→1). + // Threshold set to maxReplicas so desired = 1/max ≈ 1 (never drives above min). + // activationThreshold=0 ensures any session activates the ScaledObject. + // + // Scale-down: HPA policy removes 1 pod per cooldown period (preserves cache). + // Idle (all sessions closed + no queued requests): after cooldownPeriod → 0. + // "or vector(0)" ensures queries return 0 (not empty) when pods don't exist. + String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; + String namespace = hiveCluster.getMetadata().getNamespace(); + spec.put("triggers", List.of( + Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "llap_num_queued_requests", + "query", String.format( + "avg(hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + namespace, targetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + ), + Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_open_sessions_activation", + "query", String.format( + "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(maxReplicas), + "activationThreshold", "0" + ) + ) + )); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("keda.sh/v1alpha1") + .withKind("ScaledObject") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "llap")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-llap-scaledobject"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index c8c044d22ce9..35fcbb8ac171 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -26,6 +26,8 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.Lifecycle; +import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder; @@ -34,7 +36,9 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -81,16 +85,15 @@ protected StatefulSet desired(HiveCluster hiveCluster, envVars.addAll(spec.envVars()); } - List ports = List.of( - new ContainerPortBuilder() - .withName("management").withContainerPort(15004).build(), - new ContainerPortBuilder() - .withName("shuffle").withContainerPort(15551).build(), - new ContainerPortBuilder() - .withName("web").withContainerPort(15002).build(), - new ContainerPortBuilder() - .withName("output").withContainerPort(15003).build() - ); + List ports = new ArrayList<>(); + ports.add(new ContainerPortBuilder() + .withName("management").withContainerPort(15004).build()); + ports.add(new ContainerPortBuilder() + .withName("shuffle").withContainerPort(15551).build()); + ports.add(new ContainerPortBuilder() + .withName("web").withContainerPort(15002).build()); + ports.add(new ContainerPortBuilder() + .withName("output").withContainerPort(15003).build()); Probe readinessProbe = buildTcpProbe(15004, llap.readinessProbe(), 15, 10, 3); @@ -115,11 +118,31 @@ protected StatefulSet desired(HiveCluster hiveCluster, replaceConfMountWithSubPaths(volumeMounts, "llap-config", "llap-daemon-site.xml", "core-site.xml"); + // Add Prometheus JMX Exporter when autoscaling is enabled + AutoscalingSpec autoscaling = llap.autoscaling(); + if (autoscaling.isEnabled()) { + addJmxExporter(spec.image(), COMPONENT, + initContainers, volumeMounts, volumes, envVars, ports); + } + // Pre-compute config hash for the pod template annotation. String configHash = sha256( HadoopXmlBuilder.buildXml(HiveConfigBuilder.getLlapDaemonSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + // When autoscaling is enabled and the StatefulSet already exists, preserve the current + // replica count (managed by KEDA/HPA). On initial creation: + // - minReplicas == 0: start at 0, KEDA scales up when hs2_active_sessions > 0 + // - minReplicas > 0: start at configured replicas + boolean autoscalingEnabled = llap.autoscaling() != null && llap.autoscaling().isEnabled(); + Integer replicas = llap.replicas(); + if (autoscalingEnabled) { + int initialReplicas = llap.autoscaling().minReplicas() == 0 ? 0 : llap.replicas(); + replicas = getSecondaryResource(hiveCluster, context) + .map(s -> s.getSpec().getReplicas()) + .orElse(initialReplicas); + } + StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -127,7 +150,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(llap.replicas()) + .withReplicas(replicas) .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) @@ -159,6 +182,57 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); + // Graceful scale-down: poll JMX Exporter (port 9404) until all executors idle. + // Uses flat Prometheus text format — same metrics KEDA reads — not brittle JSON parsing. + if (autoscaling.isEnabled()) { + String preStopScript = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Waiting for LLAP executors to become idle (polling localhost:9404/metrics)...'", + "RETRIES=0", + "while true; do", + " RESPONSE=$(curl -sf http://localhost:9404/metrics)", + " if [ $? -ne 0 ]; then", + " RETRIES=$((RETRIES+1))", + " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", + " if [ $RETRIES -ge 6 ]; then", + " echo '[preStop] JMX Exporter not responding after 60s. Proceeding with shutdown.'", + " break", + " fi", + " sleep 10; continue", + " fi", + " AVAILABLE=$(echo \"$RESPONSE\" | grep '^hadoop_llapdaemon_executornumexecutorsavailable{' | awk '{print $2}')", + " TOTAL=$(echo \"$RESPONSE\" | grep '^hadoop_llapdaemon_executornumexecutors{' | awk '{print $2}')", + " if [ -z \"$AVAILABLE\" ] || [ -z \"$TOTAL\" ]; then", + " echo '[preStop] WARNING: LLAP executor metrics not found. JMX Exporter may not be configured.'", + " break", + " fi", + " if [ \"${AVAILABLE%.*}\" -ge \"${TOTAL%.*}\" ] 2>/dev/null; then", + " echo '[preStop] All executors idle. Shutting down.'", + " break", + " fi", + " echo \"[preStop] Executors available=$AVAILABLE / total=$TOTAL — waiting...\"", + " RETRIES=0", + " sleep 10", + "done"); + Lifecycle lifecycle = new LifecycleBuilder() + .withNewPreStop() + .withNewExec() + .withCommand("/bin/bash", "-c", preStopScript) + .endExec() + .endPreStop() + .build(); + statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); + statefulSet.getSpec().getTemplate().getSpec() + .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); + // Prometheus scrape annotations for JMX Exporter metrics endpoint + statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/scrape", "true"); + statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); + statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/path", "/metrics"); + } + if (spec.volumes() != null) { statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index 46a95426c969..e1f88caacb63 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -26,6 +26,8 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.Lifecycle; +import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.Volume; import io.fabric8.kubernetes.api.model.VolumeMount; @@ -36,6 +38,7 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.DatabaseConfig; import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; @@ -77,12 +80,11 @@ protected Deployment desired(HiveCluster hiveCluster, ConfigUtils.METASTORE_THRIFT_PORT_KEY, ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); - List ports = List.of( - new ContainerPortBuilder() - .withName("thrift").withContainerPort(thriftPort).build(), - new ContainerPortBuilder() - .withName("rest").withContainerPort(9001).build() - ); + List ports = new ArrayList<>(); + ports.add(new ContainerPortBuilder() + .withName("thrift").withContainerPort(thriftPort).build()); + ports.add(new ContainerPortBuilder() + .withName("rest").withContainerPort(9001).build()); Probe readinessProbe = buildTcpProbe(thriftPort, spec.metastore().readinessProbe(), 15, 10, 3); Probe livenessProbe = buildTcpProbe(thriftPort, spec.metastore().livenessProbe(), 60, 30, 5); @@ -107,6 +109,13 @@ protected Deployment desired(HiveCluster hiveCluster, replaceConfMountWithSubPaths(volumeMounts, "hive-config", "metastore-site.xml", "core-site.xml"); + // Add Prometheus JMX Exporter when autoscaling is enabled + AutoscalingSpec autoscaling = spec.metastore().autoscaling(); + if (autoscaling.isEnabled()) { + addJmxExporter(spec.image(), COMPONENT, + initContainers, volumeMounts, volumes, envVars, ports); + } + // Pre-compute config hash for the pod template annotation. // This ensures the Deployment is created with the correct hash // from the start (single ReplicaSet) and triggers rolling @@ -115,6 +124,19 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getMetastoreSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + // When autoscaling is enabled and the Deployment already exists, preserve the current + // replica count (managed by KEDA/HPA). On initial creation, start at minReplicas + // and let KEDA scale up based on load. + boolean autoscalingEnabled = spec.metastore().autoscaling() != null + && spec.metastore().autoscaling().isEnabled(); + Integer replicas = spec.metastore().replicas(); + if (autoscalingEnabled) { + int initialReplicas = Math.max(1, spec.metastore().autoscaling().minReplicas()); + replicas = getSecondaryResource(hiveCluster, context) + .map(d -> d.getSpec().getReplicas()) + .orElse(initialReplicas); + } + Deployment deployment = new DeploymentBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -122,7 +144,7 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(spec.metastore().replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() @@ -155,6 +177,57 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); + // Graceful scale-down: poll JMX Exporter (port 9404) for open_connections to drain. + // K8s removes the pod from Service Endpoints on termination, so no new requests arrive. + // Uses flat Prometheus text format — same metric KEDA reads — not brittle JSON parsing. + if (autoscaling.isEnabled()) { + String preStopScript = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Waiting for open connections to drain (polling localhost:9404/metrics)...'", + "RETRIES=0", + "while true; do", + " RESPONSE=$(curl -sf http://localhost:9404/metrics)", + " if [ $? -ne 0 ]; then", + " RETRIES=$((RETRIES+1))", + " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", + " if [ $RETRIES -ge 6 ]; then", + " echo '[preStop] JMX Exporter not responding after 30s. Proceeding with shutdown.'", + " break", + " fi", + " sleep 5; continue", + " fi", + " CONNS=$(echo \"$RESPONSE\" | grep '^hive_metastore_open_connections ' | awk '{print $2}')", + " if [ -z \"$CONNS\" ]; then", + " echo '[preStop] WARNING: hive_metastore_open_connections metric not found. JMX Exporter may not be configured.'", + " break", + " fi", + " if [ \"${CONNS%.*}\" -le 0 ] 2>/dev/null; then", + " echo '[preStop] All connections drained. Shutting down.'", + " break", + " fi", + " echo \"[preStop] hive_metastore_open_connections=$CONNS — waiting...\"", + " RETRIES=0", + " sleep 5", + "done"); + Lifecycle lifecycle = new LifecycleBuilder() + .withNewPreStop() + .withNewExec() + .withCommand("/bin/bash", "-c", preStopScript) + .endExec() + .endPreStop() + .build(); + deployment.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); + deployment.getSpec().getTemplate().getSpec() + .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); + // Prometheus scrape annotations for JMX Exporter metrics endpoint + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/scrape", "true"); + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); + deployment.getSpec().getTemplate().getMetadata().getAnnotations() + .put("prometheus.io/path", "/metrics"); + } + if (spec.volumes() != null) { deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java new file mode 100644 index 000000000000..e177e1e60138 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java @@ -0,0 +1,62 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * PodDisruptionBudget for Hive Metastore. + * Ensures at least one Metastore pod remains available during voluntary disruptions + * to prevent catalog access failures. + */ +public class MetastorePdbDependent + extends CRUDKubernetesDependentResource { + + public MetastorePdbDependent() { + super(PodDisruptionBudget.class); + } + + @Override + protected PodDisruptionBudget desired(HiveCluster hiveCluster, + Context context) { + return new PodDisruptionBudgetBuilder() + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "metastore")) + .endMetadata() + .withNewSpec() + .withMinAvailable(new IntOrString(1)) + .withNewSelector() + .withMatchLabels(Labels.selectorForComponent(hiveCluster, "metastore")) + .endSelector() + .endSpec() + .build(); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-metastore-pdb"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java new file mode 100644 index 000000000000..58263318468f --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java @@ -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.hive.kubernetes.operator.dependent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA ScaledObject for Hive Metastore autoscaling. + *

+ * Scale-up triggers (OR): + * - Open connections exceed threshold (Prometheus) + * - Pod CPU > 75% + *

+ * Scale-down triggers (AND): + * - CPU < activationValue + * - Open connections at 0 + *

+ * Cooldown: configurable (default 300s / 5 minutes) + * Guardrail: replicas should be set based on backend DB max_connections. + */ +public class MetastoreScaledObjectDependent extends HiveGenericDependentResource { + + public MetastoreScaledObjectDependent() { + super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().metastore().autoscaling(); + int maxReplicas = hiveCluster.getSpec().metastore().replicas(); + String targetName = hiveCluster.getMetadata().getName() + "-metastore"; + + // Threshold = max threads per pod (from metastore-site config or default 1000). + // KEDA divides total open_connections by threshold to determine desired replicas. + int maxThreads = ConfigUtils.getInt( + hiveCluster.getSpec().metastore().configOverrides(), + ConfigUtils.METASTORE_SERVER_MAX_THREADS_KEY, + ConfigUtils.METASTORE_SERVER_MAX_THREADS_HIVE_KEY, + ConfigUtils.METASTORE_SERVER_MAX_THREADS_DEFAULT); + + Map spec = new HashMap<>(); + spec.put("scaleTargetRef", Map.of( + "apiVersion", "apps/v1", + "kind", "Deployment", + "name", targetName + )); + spec.put("minReplicaCount", autoscaling.minReplicas()); + spec.put("maxReplicaCount", maxReplicas); + spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); + spec.put("pollingInterval", 30); + + spec.put("advanced", Map.of( + "horizontalPodAutoscalerConfig", Map.of( + "behavior", Map.of( + "scaleDown", Map.of( + "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), + "policies", List.of(Map.of( + "type", "Pods", + "value", 1, + "periodSeconds", 60 + )) + ), + "scaleUp", Map.of( + "stabilizationWindowSeconds", 120, + "policies", List.of(Map.of( + "type", "Percent", + "value", 50, + "periodSeconds", 60 + )) + ) + ) + ) + )); + + // Triggers: Prometheus for open connections + CPU (only when CPU requests are defined) + // "or vector(0)" ensures the query returns 0 (not empty) when no pods match. + List> triggers = new ArrayList<>(); + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hive_metastore_open_connections", + "query", String.format( + "sum(hive_metastore_open_connections{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName), + "threshold", String.valueOf(maxThreads), + "activationThreshold", "0" + ) + )); + if (hiveCluster.getSpec().metastore().resources() != null) { + // activationValue prevents idle JVM CPU from keeping the ScaledObject active. + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", "75", + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + } + spec.put("triggers", triggers); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("keda.sh/v1alpha1") + .withKind("ScaledObject") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "metastore")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-metastore-scaledobject"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java new file mode 100644 index 000000000000..13fc6343cad0 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java @@ -0,0 +1,61 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * PodDisruptionBudget for Tez Application Masters. + * Ensures at least one Tez AM remains available in the warm pool during voluntary disruptions. + */ +public class TezAmPdbDependent + extends CRUDKubernetesDependentResource { + + public TezAmPdbDependent() { + super(PodDisruptionBudget.class); + } + + @Override + protected PodDisruptionBudget desired(HiveCluster hiveCluster, + Context context) { + return new PodDisruptionBudgetBuilder() + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "tezam")) + .endMetadata() + .withNewSpec() + .withMinAvailable(new IntOrString(1)) + .withNewSelector() + .withMatchLabels(Labels.selectorForComponent(hiveCluster, "tezam")) + .endSelector() + .endSpec() + .build(); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-tezam-pdb"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java new file mode 100644 index 000000000000..731eb6f08d97 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java @@ -0,0 +1,200 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA ScaledObject for Tez Application Master autoscaling. + *

+ * Tez AMs run in a warm pool (StatefulSet). An unclaimed AM sits idle; + * a claimed AM actively orchestrates a query DAG and consumes CPU. + *

+ * Scale-up trigger: + * - Pod CPU > 60% across the StatefulSet (most AMs claimed and working) + *

+ * Scale-down trigger: + * - Pod CPU < 10% (many idle unclaimed AMs) + *

+ * Cooldown: configurable (default 600s / 10 minutes) + */ +public class TezAmScaledObjectDependent extends HiveGenericDependentResource { + + public TezAmScaledObjectDependent() { + super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().tezAm().autoscaling(); + int maxReplicas = hiveCluster.getSpec().tezAm().replicas(); + String targetName = hiveCluster.getMetadata().getName() + "-tezam"; + + Map spec = new HashMap<>(); + spec.put("scaleTargetRef", Map.of( + "apiVersion", "apps/v1", + "kind", "StatefulSet", + "name", targetName + )); + // KEDA requires idleReplicaCount < minReplicaCount. + // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). + // For non-zero min: just set minReplicaCount (no idle needed). + int minReplicaCount = Math.max(1, autoscaling.minReplicas()); + spec.put("minReplicaCount", minReplicaCount); + spec.put("maxReplicaCount", maxReplicas); + if (autoscaling.minReplicas() == 0) { + spec.put("idleReplicaCount", 0); + } + spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); + spec.put("pollingInterval", 5); + + spec.put("advanced", Map.of( + "horizontalPodAutoscalerConfig", Map.of( + "behavior", Map.of( + "scaleDown", Map.of( + "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), + "policies", List.of(Map.of( + "type", "Pods", + "value", 1, + "periodSeconds", 60 + )) + ), + "scaleUp", Map.of( + "stabilizationWindowSeconds", 60, + "policies", List.of(Map.of( + "type", "Pods", + "value", 2, + "periodSeconds", 30 + )) + ) + ) + ) + )); + + // Triggers: + // 1. CPU utilization — the primary proportional scaler for warm-pool Tez AMs + // (only included when container has CPU requests defined, required by KEDA) + // 2. HS2 cross-component activation: when HS2 has open sessions, + // TezAM should be available (enables wake-from-zero) + // + // When CPU IS available: CPU drives proportional scaling, HS2 trigger is activation-only + // (threshold set to maxReplicas so it never dominates the HPA calculation). + // When CPU is NOT available: tez_session_pending_tasks drives proportional scaling + // (real query demand — tasks waiting for AM slots), with HS2 sessions for activation only. + String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; + String namespace = hiveCluster.getMetadata().getNamespace(); + List> triggers = new ArrayList<>(); + if (hiveCluster.getSpec().tezAm().resources() != null) { + // CPU drives proportional scaling; activationValue prevents idle JVM CPU + // from keeping the ScaledObject permanently "active" (blocks scale-to-zero). + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", String.valueOf(autoscaling.scaleUpThreshold()), + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + // Activation-only: (sessions > bool 0) returns 0 or 1, with threshold=maxReplicas + // ensures desired = ceil(1/max) = 1 — never drives replica count above min. + // activationThreshold=0 ensures any open session wakes TezAM from zero. + // Uses hs2_open_sessions (connection-level) not hs2_active_sessions (query-level). + // "or vector(0)" ensures the query returns 0 (not empty) when HS2 has no pods. + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_open_sessions_activation", + "query", String.format( + "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(maxReplicas), + "activationThreshold", "0" + ) + )); + } else { + // No CPU available: use tez_session_pending_tasks for proportional scaling. + // This metric reflects real query demand (tasks waiting for AM slots), unlike + // hs2_open_sessions which includes zombie/idle sessions from ungracefully closed clients. + // Threshold: scaleUpThreshold interpreted as pending-tasks-per-AM (default 60 when + // using CPU mode, but for pending tasks a lower value like 5-10 is recommended). + // "or vector(0)" ensures the query returns 0 when HS2 has no pods. + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "tez_session_pending_tasks", + "query", String.format( + "sum(tez_session_pending_tasks{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + )); + // Activation-only: (sessions > bool 0) returns 0 or 1, with threshold=maxReplicas + // ensures desired = ceil(1/max) = 1 — never drives replica count above min. + // activationThreshold=0 ensures any open session wakes TezAM from zero. + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_open_sessions_activation", + "query", String.format( + "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(maxReplicas), + "activationThreshold", "0" + ) + )); + } + spec.put("triggers", triggers); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("keda.sh/v1alpha1") + .withKind("ScaledObject") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, "tezam")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-tezam-scaledobject"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index 5cc7a3f800f3..56e60ca10403 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -23,7 +23,11 @@ import java.util.Map; import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerPort; +import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.Lifecycle; +import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder; import io.javaoperatorsdk.operator.api.reconciler.Context; @@ -31,6 +35,7 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.TezAmSpec; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; @@ -107,12 +112,20 @@ protected StatefulSet desired(HiveCluster hiveCluster, .endPersistentVolumeClaim() .build()); + List ports = new ArrayList<>(); List initContainers = new ArrayList<>(); addExternalJars(spec.image(), spec.externalJars(), initContainers, volumeMounts, volumes, envVars); replaceConfMountWithSubPaths(volumeMounts, "hive-config", "hive-site.xml", "tez-site.xml", "core-site.xml"); + // Add Prometheus JMX Exporter when autoscaling is enabled + AutoscalingSpec autoscaling = tezAm.autoscaling(); + if (autoscaling.isEnabled()) { + addJmxExporter(spec.image(), COMPONENT, + initContainers, volumeMounts, volumes, envVars, ports); + } + // Pre-compute config hash for the pod template annotation. // TezAM uses the same ConfigMaps as HS2 (hive-site.xml + tez-site.xml + core-site.xml). String configHash = sha256( @@ -120,6 +133,19 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + // When autoscaling is enabled and the StatefulSet already exists, preserve the current + // replica count (managed by KEDA/HPA). On initial creation: + // - minReplicas == 0: start at 0, KEDA scales up when hs2_active_sessions > 0 + // - minReplicas > 0: start at configured replicas + boolean autoscalingEnabled = tezAm.autoscaling() != null && tezAm.autoscaling().isEnabled(); + Integer replicas = tezAm.replicas(); + if (autoscalingEnabled) { + int initialReplicas = tezAm.autoscaling().minReplicas() == 0 ? 0 : tezAm.replicas(); + replicas = getSecondaryResource(hiveCluster, context) + .map(s -> s.getSpec().getReplicas()) + .orElse(initialReplicas); + } + StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -127,7 +153,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(tezAm.replicas()) + .withReplicas(replicas) .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) @@ -145,6 +171,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) + .withPorts(ports) .withResources(buildResources(tezAm.resources())) .withVolumeMounts(volumeMounts) .endContainer() @@ -157,6 +184,50 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); + // Graceful scale-down: poll JMX Exporter (port 9404) for DAGsRunning to reach 0. + // K8s removes the pod from Service Endpoints, so HS2 won't assign new DAGs to this AM. + // We read from the same Prometheus-format endpoint that KEDA uses — flat text, not brittle JSON. + if (autoscaling.isEnabled()) { + String preStopScript = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Waiting for active DAGs to complete (polling localhost:9404/metrics)...'", + "RETRIES=0", + "while true; do", + " RESPONSE=$(curl -sf http://localhost:9404/metrics)", + " if [ $? -ne 0 ]; then", + " RETRIES=$((RETRIES+1))", + " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", + " if [ $RETRIES -ge 6 ]; then", + " echo '[preStop] JMX Exporter not responding after 60s. Proceeding with shutdown.'", + " break", + " fi", + " sleep 10; continue", + " fi", + " DAGS=$(echo \"$RESPONSE\" | grep '^tez_am_dagsrunning ' | awk '{print $2}')", + " if [ -z \"$DAGS\" ]; then", + " echo '[preStop] WARNING: tez_am_dagsrunning metric not found. JMX Exporter may not be configured.'", + " break", + " fi", + " if [ \"${DAGS%.*}\" -le 0 ] 2>/dev/null; then", + " echo '[preStop] No active DAGs. Safe to terminate Tez AM.'", + " break", + " fi", + " echo \"[preStop] tez_am_dagsrunning=$DAGS — waiting...\"", + " RETRIES=0", + " sleep 10", + "done"); + Lifecycle lifecycle = new LifecycleBuilder() + .withNewPreStop() + .withNewExec() + .withCommand("/bin/bash", "-c", preStopScript) + .endExec() + .endPreStop() + .build(); + statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); + statefulSet.getSpec().getTemplate().getSpec() + .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); + } + if (spec.volumes() != null) { statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java new file mode 100644 index 000000000000..bf14dac91e54 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java @@ -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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Activation condition for HiveServer2 autoscaling dependent resources. + * Returns true only when spec.hiveServer2.autoscaling.enabled is true. + */ +public class HiveServer2AutoscalingCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + return primary.getSpec().hiveServer2().autoscaling().isEnabled(); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java new file mode 100644 index 000000000000..9c01942e4a1c --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java @@ -0,0 +1,44 @@ +/* + * 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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; + +/** + * Activation condition for HiveServer2 Prometheus-based ScaledObject. + * Returns true when autoscaling is enabled AND minReplicas > 0. + * When minReplicas == 0, the HTTPScaledObject is used instead (scale-to-zero). + */ +public class HiveServer2MetricScalingCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + AutoscalingSpec autoscaling = primary.getSpec().hiveServer2().autoscaling(); + return autoscaling.isEnabled() && autoscaling.minReplicas() > 0; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java index a36002dbf886..81f07269e9c9 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java @@ -41,7 +41,14 @@ public boolean isMet( return true; } - int desiredReplicas = primary.getSpec().metastore().replicas(); + // When autoscaling is enabled, wait for minReplicas (KEDA manages scaling beyond that). + // Without autoscaling, wait for all configured replicas. + int desiredReplicas; + if (primary.getSpec().metastore().autoscaling().isEnabled()) { + desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); + } else { + desiredReplicas = primary.getSpec().metastore().replicas(); + } return context.getSecondaryResources(Deployment.class).stream() .filter(d -> d.getMetadata().getName().equals(primary.getMetadata().getName() + "-metastore")) .findFirst() diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java new file mode 100644 index 000000000000..7ae91b8f7b8f --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java @@ -0,0 +1,44 @@ +/* + * 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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; + +/** + * Activation condition for HiveServer2 scale-to-zero (HTTPScaledObject). + * Returns true when autoscaling is enabled AND minReplicas == 0. + * Requires the KEDA HTTP Add-on to be installed in the cluster. + */ +public class HiveServer2ScaleToZeroCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + AutoscalingSpec autoscaling = primary.getSpec().hiveServer2().autoscaling(); + return autoscaling.isEnabled() && autoscaling.minReplicas() == 0; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java new file mode 100644 index 000000000000..f4e097786b08 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java @@ -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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Activation condition for LLAP autoscaling dependent resources. + * Returns true only when spec.llap.enabled is true and spec.llap.autoscaling.enabled is true. + */ +public class LlapAutoscalingCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + return primary.getSpec().llap().isEnabled() + && primary.getSpec().llap().autoscaling().isEnabled(); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java new file mode 100644 index 000000000000..a0ac83d8a423 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java @@ -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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Activation condition for Metastore autoscaling dependent resources. + * Returns true only when spec.metastore.autoscaling.enabled is true and metastore is managed. + */ +public class MetastoreAutoscalingCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + return primary.getSpec().metastore().isEnabled() + && primary.getSpec().metastore().autoscaling().isEnabled(); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java index 7b3169f32043..58885c6e8865 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java @@ -39,7 +39,14 @@ public boolean isMet( if (!primary.getSpec().metastore().isEnabled()) { return true; } - int desiredReplicas = primary.getSpec().metastore().replicas(); + // When autoscaling is enabled, wait for minReplicas (KEDA manages scaling beyond that). + // Without autoscaling, wait for all configured replicas. + int desiredReplicas; + if (primary.getSpec().metastore().autoscaling().isEnabled()) { + desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); + } else { + desiredReplicas = primary.getSpec().metastore().replicas(); + } return dependentResource.getSecondaryResource(primary, context) .map(deployment -> deployment.getStatus() != null && deployment.getStatus().getReadyReplicas() != null diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java new file mode 100644 index 000000000000..a2ed23cbbadc --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java @@ -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.hive.kubernetes.operator.dependent.condition; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Activation condition for Tez AM autoscaling dependent resources. + * Returns true only when spec.tezAm.enabled is true and spec.tezAm.autoscaling.enabled is true. + */ +public class TezAmAutoscalingCondition + implements Condition { + + @Override + public boolean isMet( + DependentResource dependentResource, + HiveCluster primary, + Context context) { + return primary.getSpec().tezAm().isEnabled() + && primary.getSpec().tezAm().autoscaling().isEnabled(); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java new file mode 100644 index 000000000000..eb0980fb1a1d --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -0,0 +1,62 @@ +/* + * 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.hive.kubernetes.operator.model.spec; + +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import io.fabric8.generator.annotation.Default; + +/** Autoscaling configuration for a Hive component. Uses KEDA ScaledObjects for metric-based scaling. */ +public record AutoscalingSpec( + @JsonPropertyDescription("Whether autoscaling is enabled for this component") + @Default("false") + Boolean enabled, + @JsonPropertyDescription("Minimum number of replicas (floor for scale-down). " + + "Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero)") + @Default("0") + Integer minReplicas, + @JsonPropertyDescription("Threshold that triggers scale-up (component-specific: " + + "sessions for HS2, queue depth for LLAP, CPU% for TezAM with resources, " + + "pending tasks per AM for TezAM without resources)") + @Default("80") + Integer scaleUpThreshold, + @JsonPropertyDescription("Percentage threshold that triggers scale-down " + + "(all conditions must be met: metric below threshold AND CPU below threshold)") + @Default("20") + Integer scaleDownThreshold, + @JsonPropertyDescription("Cooldown period in seconds after a scaling event before another can occur") + @Default("600") + Integer cooldownSeconds, + @JsonPropertyDescription("Maximum time in seconds to wait for graceful drain " + + "during scale-down before the pod is forcibly terminated") + @Default("300") + Integer gracePeriodSeconds) { + + public AutoscalingSpec { + enabled = enabled != null ? enabled : false; + minReplicas = minReplicas != null ? minReplicas : 0; + scaleUpThreshold = scaleUpThreshold != null ? scaleUpThreshold : 80; + scaleDownThreshold = scaleDownThreshold != null ? scaleDownThreshold : 20; + cooldownSeconds = cooldownSeconds != null ? cooldownSeconds : 600; + gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 300; + } + + public boolean isEnabled() { + return enabled; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index 78164fb32de6..993b452ba4b4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -51,7 +51,9 @@ public record HiveServer2Spec( @JsonPropertyDescription("Readiness probe configuration") ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") - ProbeSpec livenessProbe) { + ProbeSpec livenessProbe, + @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + AutoscalingSpec autoscaling) { public HiveServer2Spec { replicas = replicas != null ? replicas : 1; @@ -59,5 +61,7 @@ public record HiveServer2Spec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); + autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( + false, 0, 80, 20, 600, 300); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index 17ff5967ff9a..34cfc872f189 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -55,7 +55,9 @@ public record LlapSpec( @JsonPropertyDescription("LLAP service hosts identifier for ZooKeeper registration") String serviceHosts, @JsonPropertyDescription("Readiness probe configuration") - ProbeSpec readinessProbe) { + ProbeSpec readinessProbe, + @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + AutoscalingSpec autoscaling) { public LlapSpec { replicas = replicas != null ? replicas : 1; @@ -65,6 +67,8 @@ public record LlapSpec( serviceHosts = serviceHosts != null ? serviceHosts : "@llap0"; extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); + autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( + false, 0, 1, 0, 900, 600); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 307c17221ee7..51dd6dea7259 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -56,7 +56,9 @@ public record MetastoreSpec( @JsonPropertyDescription("Readiness probe configuration") ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") - ProbeSpec livenessProbe) { + ProbeSpec livenessProbe, + @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + AutoscalingSpec autoscaling) { public MetastoreSpec { replicas = replicas != null ? replicas : 1; @@ -66,6 +68,8 @@ public record MetastoreSpec( enabled = enabled != null ? enabled : true; extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); + autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( + false, 1, 75, 30, 300, 60); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index a0494c2c5e73..c973145b0080 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -52,7 +52,9 @@ public record TezAmSpec( String scratchStorageSize, @JsonPropertyDescription("StorageClass for the shared scratch PVC. " + "Must support ReadWriteMany access. If null, uses cluster default.") - String scratchStorageClassName) { + String scratchStorageClassName, + @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + AutoscalingSpec autoscaling) { public TezAmSpec { replicas = replicas != null ? replicas : 1; @@ -60,6 +62,8 @@ public record TezAmSpec( scratchStorageSize = scratchStorageSize != null ? scratchStorageSize : "1Gi"; extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); + autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( + false, 0, 60, 10, 600, 120); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 20332cb4127c..e621f7065a54 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -40,22 +40,37 @@ import org.apache.hive.kubernetes.operator.dependent.HadoopConfigMapDependent; import org.apache.hive.kubernetes.operator.dependent.HiveServer2ConfigMapDependent; import org.apache.hive.kubernetes.operator.dependent.HiveServer2DeploymentDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServer2PdbDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServer2HttpScaledObjectDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServer2ScaledObjectDependent; import org.apache.hive.kubernetes.operator.dependent.HiveServer2ServiceDependent; import org.apache.hive.kubernetes.operator.dependent.LlapConfigMapDependent; +import org.apache.hive.kubernetes.operator.dependent.LlapPdbDependent; +import org.apache.hive.kubernetes.operator.dependent.LlapScaledObjectDependent; import org.apache.hive.kubernetes.operator.dependent.LlapServiceDependent; import org.apache.hive.kubernetes.operator.dependent.LlapStatefulSetDependent; import org.apache.hive.kubernetes.operator.dependent.MetastoreConfigMapDependent; import org.apache.hive.kubernetes.operator.dependent.MetastoreDeploymentDependent; +import org.apache.hive.kubernetes.operator.dependent.MetastorePdbDependent; +import org.apache.hive.kubernetes.operator.dependent.MetastoreScaledObjectDependent; import org.apache.hive.kubernetes.operator.dependent.MetastoreServiceDependent; import org.apache.hive.kubernetes.operator.dependent.SchemaInitJobDependent; import org.apache.hive.kubernetes.operator.dependent.ScratchPvcDependent; +import org.apache.hive.kubernetes.operator.dependent.TezAmPdbDependent; +import org.apache.hive.kubernetes.operator.dependent.TezAmScaledObjectDependent; import org.apache.hive.kubernetes.operator.dependent.TezAmServiceDependent; import org.apache.hive.kubernetes.operator.dependent.TezAmStatefulSetDependent; +import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2AutoscalingCondition; +import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2MetricScalingCondition; import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2Precondition; +import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2ScaleToZeroCondition; +import org.apache.hive.kubernetes.operator.dependent.condition.LlapAutoscalingCondition; import org.apache.hive.kubernetes.operator.dependent.condition.LlapEnabledCondition; +import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreAutoscalingCondition; import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreEnabledCondition; import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreReadyCondition; import org.apache.hive.kubernetes.operator.dependent.condition.SchemaJobCompletedCondition; +import org.apache.hive.kubernetes.operator.dependent.condition.TezAmAutoscalingCondition; import org.apache.hive.kubernetes.operator.dependent.condition.TezAmEnabledCondition; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterStatus; @@ -102,7 +117,27 @@ @Dependent(name = "tezam-service", type = TezAmServiceDependent.class, activationCondition = TezAmEnabledCondition.class), @Dependent(name = "tezam-statefulset", type = TezAmStatefulSetDependent.class, dependsOn = {"hiveserver2-configmap", - "hadoop-configmap", "tezam-service", "scratch-pvc"}, activationCondition = TezAmEnabledCondition.class)}) + "hadoop-configmap", "tezam-service", "scratch-pvc"}, activationCondition = TezAmEnabledCondition.class), + // --- Autoscaling: KEDA ScaledObjects (conditional) --- + @Dependent(name = "hs2-scaledobject", type = HiveServer2ScaledObjectDependent.class, dependsOn = { + "hiveserver2-deployment"}, activationCondition = HiveServer2MetricScalingCondition.class), + @Dependent(name = "hs2-httpso", type = HiveServer2HttpScaledObjectDependent.class, dependsOn = { + "hiveserver2-deployment"}, activationCondition = HiveServer2ScaleToZeroCondition.class), + @Dependent(name = "metastore-scaledobject", type = MetastoreScaledObjectDependent.class, dependsOn = { + "metastore-deployment"}, activationCondition = MetastoreAutoscalingCondition.class), + @Dependent(name = "llap-scaledobject", type = LlapScaledObjectDependent.class, dependsOn = { + "llap-statefulset"}, activationCondition = LlapAutoscalingCondition.class), + @Dependent(name = "tezam-scaledobject", type = TezAmScaledObjectDependent.class, dependsOn = { + "tezam-statefulset"}, activationCondition = TezAmAutoscalingCondition.class), + // --- Autoscaling: PodDisruptionBudgets (conditional) --- + @Dependent(name = "hs2-pdb", type = HiveServer2PdbDependent.class, dependsOn = { + "hiveserver2-deployment"}, activationCondition = HiveServer2AutoscalingCondition.class), + @Dependent(name = "metastore-pdb", type = MetastorePdbDependent.class, dependsOn = { + "metastore-deployment"}, activationCondition = MetastoreAutoscalingCondition.class), + @Dependent(name = "llap-pdb", type = LlapPdbDependent.class, dependsOn = { + "llap-statefulset"}, activationCondition = LlapAutoscalingCondition.class), + @Dependent(name = "tezam-pdb", type = TezAmPdbDependent.class, dependsOn = { + "tezam-statefulset"}, activationCondition = TezAmAutoscalingCondition.class)}) public class HiveClusterReconciler implements Reconciler { private static final Logger LOG = LoggerFactory.getLogger(HiveClusterReconciler.class); @@ -172,9 +207,13 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // Metastore status boolean metastoreReady; if (resource.getSpec().metastore().isEnabled()) { + // When autoscaling, desired = minReplicas (KEDA manages beyond that) + int metastoreDesired = resource.getSpec().metastore().autoscaling().isEnabled() + ? Math.max(1, resource.getSpec().metastore().autoscaling().minReplicas()) + : resource.getSpec().metastore().replicas(); ComponentStatus metastoreStatus = buildComponentStatus(context, Deployment.class, resource.getMetadata().getName() + "-metastore", - resource.getSpec().metastore().replicas(), + metastoreDesired, d -> d.getStatus() != null && d.getStatus().getReadyReplicas() != null ? d.getStatus().getReadyReplicas() : 0); @@ -192,15 +231,17 @@ private HiveClusterStatus buildStatus(HiveCluster resource, existingConditions)); } - // HiveServer2 status + // HiveServer2 status — when scale-to-zero, 0/0 is a valid "ready" state (idle) + int hs2Desired = resource.getSpec().hiveServer2().autoscaling().isEnabled() + ? resource.getSpec().hiveServer2().autoscaling().minReplicas() + : resource.getSpec().hiveServer2().replicas(); ComponentStatus hs2Status = buildComponentStatus(context, Deployment.class, resource.getMetadata().getName() + "-hiveserver2", - resource.getSpec().hiveServer2().replicas(), + hs2Desired, d -> d.getStatus() != null && d.getStatus().getReadyReplicas() != null ? d.getStatus().getReadyReplicas() : 0); status.setHiveServer2(hs2Status); - boolean hs2Ready = - hs2Status.getReadyReplicas() >= hs2Status.getDesiredReplicas() && hs2Status.getDesiredReplicas() > 0; + boolean hs2Ready = hs2Status.getReadyReplicas() >= hs2Status.getDesiredReplicas(); conditions.add(buildCondition("HiveServer2Ready", hs2Ready ? "True" : "False", hs2Ready ? "DeploymentReady" : "DeploymentNotReady", hs2Ready ? "HiveServer2 is ready" : "HiveServer2 not yet ready", @@ -208,17 +249,23 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // LLAP status (optional) if (resource.getSpec().llap().isEnabled()) { + int llapDesired = resource.getSpec().llap().autoscaling().isEnabled() + ? resource.getSpec().llap().autoscaling().minReplicas() + : resource.getSpec().llap().replicas(); status.setLlap(buildComponentStatus(context, StatefulSet.class, resource.getMetadata().getName() + "-llap", - resource.getSpec().llap().replicas(), + llapDesired, s -> s.getStatus() != null && s.getStatus().getReadyReplicas() != null ? s.getStatus().getReadyReplicas() : 0)); } // TezAM status (optional) if (resource.getSpec().tezAm().isEnabled()) { + int tezAmDesired = resource.getSpec().tezAm().autoscaling().isEnabled() + ? resource.getSpec().tezAm().autoscaling().minReplicas() + : resource.getSpec().tezAm().replicas(); status.setTezAm(buildComponentStatus(context, StatefulSet.class, resource.getMetadata().getName() + "-tezam", - resource.getSpec().tezAm().replicas(), + tezAmDesired, s -> s.getStatus() != null && s.getStatus().getReadyReplicas() != null ? s.getStatus().getReadyReplicas() : 0)); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 0f86201817e7..2e641cdb4c1c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -73,14 +73,34 @@ private ConfigUtils() { public static final String HIVE_LLAP_DAEMON_NUM_EXECUTORS_KEY = "hive.llap.daemon.num.executors"; + public static final String METASTORE_SERVER_MAX_THREADS_KEY = "metastore.server.max.threads"; + public static final String METASTORE_SERVER_MAX_THREADS_HIVE_KEY = "hive.metastore.server.max.threads"; + public static final int METASTORE_SERVER_MAX_THREADS_DEFAULT = 1000; + public static final String HIVE_METASTORE_URIS_KEY = "hive.metastore.uris"; public static final String HIVE_SERVER2_THRIFT_PORT_KEY = "hive.server2.thrift.port"; public static final int HIVE_SERVER2_THRIFT_PORT_DEFAULT = 10000; + public static final String HIVE_SERVER2_THRIFT_HTTP_PORT_KEY = "hive.server2.thrift.http.port"; + public static final int HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT = 10001; + + public static final String HIVE_SERVER2_THRIFT_HTTP_PATH_KEY = "hive.server2.thrift.http.path"; + public static final String HIVE_SERVER2_THRIFT_HTTP_PATH_DEFAULT = "cliservice"; + + public static final String HIVE_SERVER2_TRANSPORT_MODE_KEY = "hive.server2.transport.mode"; + public static final String HIVE_SERVER2_TRANSPORT_MODE_DEFAULT = "http"; + public static final String HIVE_SERVER2_WEBUI_PORT_KEY = "hive.server2.webui.port"; public static final int HIVE_SERVER2_WEBUI_PORT_DEFAULT = 10002; + /** Port for the Prometheus JMX Exporter agent (serves /metrics in text format). */ + public static final int PROMETHEUS_JMX_EXPORTER_PORT = 9404; + + /** Default URL for the Prometheus JMX Exporter javaagent JAR. */ + public static final String JMX_EXPORTER_JAR_URL = + "https://repo1.maven.org/maven2/io/prometheus/jmx/jmx_prometheus_javaagent/1.0.1/jmx_prometheus_javaagent-1.0.1.jar"; + public static final String TEZ_AM_SESSION_MODE_KEY = "tez.am.mode.session"; public static final String TEZ_IGNORE_LIB_URIS_KEY = "tez.ignore.lib.uris"; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java index 5db24e95d3f3..7baaf2afc34f 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java @@ -60,6 +60,12 @@ public static Map getHiveServer2HiveSite( props.put(ConfigUtils.HIVE_METASTORE_WAREHOUSE_KEY, spec.metastore().warehouseDir()); props.put(ConfigUtils.HIVE_SERVER2_ENABLE_DOAS_KEY, "false"); + props.put(ConfigUtils.HIVE_SERVER2_TRANSPORT_MODE_KEY, + ConfigUtils.HIVE_SERVER2_TRANSPORT_MODE_DEFAULT); + props.put(ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + String.valueOf(ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT)); + props.put(ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PATH_KEY, + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PATH_DEFAULT); props.put(ConfigUtils.HIVE_TEZ_EXEC_INPLACE_PROGRESS_KEY, "false"); props.put(ConfigUtils.HIVE_TEZ_EXEC_SUMMARY_KEY, "true"); props.put(ConfigUtils.HIVE_JAR_DIRECTORY_KEY, "/tmp"); @@ -95,6 +101,14 @@ public static Map getHiveServer2HiveSite( props.put("mapreduce.framework.name", "local"); } + // Enable JMX metrics when autoscaling is active. + // The Prometheus JMX Exporter agent (added by the operator) reads JMX MBeans + // and exposes them in Prometheus text format at /metrics on the metrics port. + if (spec.hiveServer2().autoscaling().isEnabled()) { + props.put("hive.server2.metrics.enabled", "true"); + props.put("hive.server2.metrics.reporter", "JMX"); + } + if (spec.hiveServer2().configOverrides() != null) { props.putAll(spec.hiveServer2().configOverrides()); } @@ -165,6 +179,14 @@ public static Map getMetastoreSite(HiveClusterSpec spec) { } } + // Enable JMX metrics when autoscaling is active. + // The Prometheus JMX Exporter agent reads JMX MBeans and exposes them + // in Prometheus text format at /metrics on the metrics port. + if (metastore.autoscaling().isEnabled()) { + props.put("metastore.metrics.enabled", "true"); + props.put("metastore.metrics.reporter", "JMX"); + } + if (metastore.configOverrides() != null) { props.putAll(metastore.configOverrides()); } From 54693dbf39571b8d9393288da4f45671efeb281d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 26 May 2026 20:21:59 +0530 Subject: [PATCH 02/22] Fix Scaling HMS & Refactor --- packaging/src/kubernetes/README.md | 247 ++++++------ .../hive-operator/templates/clusterrole.yaml | 2 +- .../kubernetes/operator/HiveOperatorMain.java | 15 +- .../dependent/HadoopConfigMapDependent.java | 67 ---- .../dependent/HiveConfigMapDependent.java | 153 ++++++++ .../dependent/HiveDependentResource.java | 279 ++++++++++++-- .../HiveGenericDependentResource.java | 88 +++++ .../operator/dependent/HivePdbDependent.java | 103 +++++ .../dependent/HiveScaledObjectDependent.java | 360 ++++++++++++++++++ .../HiveServer2ConfigMapDependent.java | 72 ---- .../HiveServer2DeploymentDependent.java | 111 ++---- .../HiveServer2HttpScaledObjectDependent.java | 5 +- .../HiveServer2InterceptorRouteDependent.java | 116 ++++++ .../dependent/HiveServer2PdbDependent.java | 62 --- .../HiveServer2ScaledObjectDependent.java | 149 -------- .../HiveServer2ServiceDependent.java | 87 ----- .../dependent/HiveServiceDependent.java | 165 ++++++++ .../dependent/LlapConfigMapDependent.java | 68 ---- .../operator/dependent/LlapPdbDependent.java | 62 --- .../dependent/LlapScaledObjectDependent.java | 158 -------- .../dependent/LlapServiceDependent.java | 77 ---- .../dependent/LlapStatefulSetDependent.java | 115 ++---- .../MetastoreConfigMapDependent.java | 67 ---- .../MetastoreDeploymentDependent.java | 103 ++--- .../dependent/MetastorePdbDependent.java | 62 --- .../MetastoreScaledObjectDependent.java | 152 -------- .../dependent/MetastoreServiceDependent.java | 75 ---- .../dependent/SchemaInitJobDependent.java | 6 + .../dependent/ScratchPvcDependent.java | 6 + .../operator/dependent/TezAmPdbDependent.java | 61 --- .../dependent/TezAmScaledObjectDependent.java | 200 ---------- .../dependent/TezAmServiceDependent.java | 62 --- .../dependent/TezAmStatefulSetDependent.java | 97 ++--- .../HiveServer2AutoscalingCondition.java | 41 -- .../HiveServer2MetricScalingCondition.java | 44 --- .../condition/HiveServer2Precondition.java | 60 --- .../HiveServer2ScaleToZeroCondition.java | 44 --- .../condition/LlapAutoscalingCondition.java | 42 -- .../condition/LlapEnabledCondition.java | 41 -- .../MetastoreAutoscalingCondition.java | 42 -- .../condition/MetastoreEnabledCondition.java | 39 -- .../condition/MetastoreReadyCondition.java | 56 --- .../SchemaJobCompletedCondition.java | 48 --- .../condition/TezAmAutoscalingCondition.java | 42 -- .../condition/TezAmEnabledCondition.java | 41 -- .../operator/model/HiveClusterSpec.java | 8 + .../operator/model/spec/TezAmSpec.java | 2 +- .../reconciler/HiveClusterReconciler.java | 182 ++++----- .../operator/reconciler/HiveWorkflowSpec.java | 290 ++++++++++++++ ...torMapJoinOuterGenerateResultOperator.java | 1 - 50 files changed, 1893 insertions(+), 2582 deletions(-) delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HadoopConfigMapDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ConfigMapDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapConfigMapDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapServiceDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreConfigMapDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreServiceDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmServiceDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapEnabledCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreEnabledCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/SchemaJobCompletedCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmEnabledCondition.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index e9ccc62d949a..ac2a4d0e9584 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -555,9 +555,11 @@ kubectl get pods -n keda -l app=keda-add-ons-http-interceptor-proxy # Expected: keda-add-ons-http-interceptor-proxy-... Running ``` -> **Note:** The HTTP Add-on is required when `minReplicas: 0`. It places an interceptor -> proxy in the traffic path that detects incoming requests when HS2 has zero pods, -> automatically scaling HS2 up and holding the request until a pod is ready. +> **Note:** The HTTP Add-on is required when `minReplicas: 0`. The operator creates +> an `InterceptorRoute` CRD that configures the interceptor proxy to route traffic +> to HS2. When HS2 has zero pods, the interceptor holds incoming requests and triggers +> scale-up via an `external-push` trigger on the HS2 ScaledObject. The first request +> takes ~30-60s while the pod starts. **For Prometheus-based triggers** (HS2, HMS, LLAP), install Prometheus: @@ -597,21 +599,25 @@ helm install prometheus prometheus-community/prometheus \ | Component | Scale-Up Trigger | Scale-Down Trigger | Cooldown | Native Metric | |-----------|-----------------|-------------------|----------|---------------| -| **HiveServer2** | `hs2_active_sessions` > 80% of max **OR** CPU > 75% | `hs2_open_sessions` < 20% **AND** CPU < 30% | 10 min | `hs2_open_sessions`, `hs2_active_sessions` | -| **Metastore** | `api_get_partitions` rate spike **OR** CPU > 75% (2 min) | CPU < 30% **AND** API rate flat | 5 min | `api_get_partitions`, `open_connections` | -| **LLAP** | `NumQueuedRequests` > 0 for 1 min | `NumExecutorsAvailable == NumExecutors` (idle) | 15 min | `NumQueuedRequests`, `NumExecutorsAvailable` | -| **Tez AM** (with CPU resources) | Pod CPU > 60% (pool is busy) | Pod CPU < 10% (pool is idle) | 10 min | Standard K8s CPU | -| **Tez AM** (without CPU resources) | `tez_session_pending_tasks` > threshold | No pending tasks for cooldown | 10 min | `tez_session_pending_tasks` | +| **HiveServer2** | `hs2_open_sessions` > scaleUpThreshold **OR** CPU > 75% | Sessions below threshold **AND** CPU below scaleDownThreshold | 5 min | `hs2_open_sessions` | +| **Metastore** | `hive_metastore_open_connections` > scaleUpThreshold **OR** CPU > 75% | Connections below threshold **AND** CPU below scaleDownThreshold | 5 min | `hive_metastore_open_connections` | +| **LLAP** | Total busy slots > scaleUpThreshold (queued + busy executors) | All executors idle + no HS2 sessions | 15 min | `NumQueuedRequests`, `NumExecutorsConfigured`, `NumExecutorsAvailable` | +| **Tez AM** (with CPU resources) | Pod CPU > scaleUpThreshold% | Pod CPU < scaleDownThreshold% + no HS2 sessions | 5 min | Standard K8s CPU | +| **Tez AM** (without CPU resources) | `tez_session_pending_tasks` > scaleUpThreshold | No pending tasks + no HS2 sessions | 5 min | `tez_session_pending_tasks` | ### Scale-to-Zero Architecture When `minReplicas: 0` is configured (default for HS2, LLAP, TezAM), the cluster -scales down to zero pods when completely idle: +scales down to zero pods when completely idle. The operator uses a **unified +ScaledObject + InterceptorRoute** architecture — a single KEDA ScaledObject per +component handles both Prometheus-based scaling and wake-from-zero, while an +`InterceptorRoute` (from the KEDA HTTP Add-on) provides routing-only configuration +without creating a conflicting second ScaledObject. ``` Scale-to-Zero (Idle Detection) - 1. No active sessions/queries for cooldownPeriod seconds + 1. No open sessions/queries for cooldownPeriod seconds → KEDA detects all triggers inactive → scales HS2 to 0 (idleReplicaCount) @@ -627,7 +633,7 @@ scales down to zero pods when completely idle: Wake-from-Zero (with KEDA HTTP Add-on) 1. Beeline connects → KEDA HTTP interceptor proxy queues the - request and triggers HS2 scale-up (0 → 1) + request and triggers HS2 scale-up via external-push trigger 2. HS2 pod starts, reports hs2_open_sessions > 0 to Prometheus @@ -639,10 +645,19 @@ scales down to zero pods when completely idle: ``` +The HS2 ScaledObject combines three trigger types in a single resource: +- **Prometheus trigger** (`hs2_open_sessions`) — session-aware scaling +- **CPU trigger** — load-based scaling when resources are configured +- **external-push trigger** — wake-from-zero via the KEDA HTTP Add-on interceptor + +The `InterceptorRoute` CRD (`http.keda.sh/v1beta1`) configures only the interceptor +routing (host matching, backend target) without auto-creating a ScaledObject — this +avoids the dual-HPA conflict that `HTTPScaledObject` would cause. + > **Important:** Automatic wake-from-zero requires the KEDA HTTP Add-on. Traffic > must flow through the interceptor proxy (via Ingress or port-forward). Without the > HTTP Add-on, HS2 must be manually woken (`kubectl scale deployment/hive-hiveserver2 --replicas=1`). -> LLAP and TezAM wake automatically once HS2 reports active sessions. See +> LLAP and TezAM wake automatically once HS2 reports open sessions. See > [Connect to HiveServer2 > Connecting with Scale-to-Zero](#connecting-with-scale-to-zero-minreplicas--0) > for setup instructions. @@ -650,7 +665,7 @@ scales down to zero pods when completely idle: | Component | minReplicas | Scale-to-Zero Trigger | Wake Trigger | |-----------|-------------|----------------------|--------------| -| **HS2** | 0 | `hs2_active_sessions = 0` for cooldown | HTTP request via KEDA interceptor (or manual) | +| **HS2** | 0 | `hs2_open_sessions = 0` for cooldown | HTTP request via KEDA interceptor (`external-push`) | | **HMS** | 1 | Never (always running) | N/A | | **LLAP** | 0 | `hs2_open_sessions = 0` for cooldown | `hs2_open_sessions > 0` (cross-component) | | **TezAM** | 0 | `hs2_open_sessions = 0` + no pending tasks | `hs2_open_sessions > 0` (cross-component) | @@ -659,6 +674,9 @@ scales down to zero pods when completely idle: **CLI (with Ozone storage backend):** +Each component has sensible per-component defaults (see [Configuration Reference](#configuration-reference)). +Only `enabled=true` is needed to turn on autoscaling: + ```bash helm install hive ./helm/hive-operator \ --set cluster.database.type=postgres \ @@ -679,29 +697,15 @@ helm install hive ./helm/hive-operator \ --set 'cluster.storage.envVars[2].name=AWS_SECRET_ACCESS_KEY' \ --set 'cluster.storage.envVars[2].value=ozone' \ --set cluster.hiveServer2.autoscaling.enabled=true \ - --set cluster.hiveServer2.autoscaling.minReplicas=0 \ - --set cluster.hiveServer2.autoscaling.scaleUpThreshold=80 \ - --set cluster.hiveServer2.autoscaling.cooldownSeconds=600 \ - --set cluster.hiveServer2.autoscaling.gracePeriodSeconds=300 \ --set cluster.metastore.autoscaling.enabled=true \ - --set cluster.metastore.autoscaling.minReplicas=1 \ - --set cluster.metastore.autoscaling.cooldownSeconds=300 \ - --set cluster.metastore.autoscaling.gracePeriodSeconds=60 \ --set cluster.llap.autoscaling.enabled=true \ - --set cluster.llap.autoscaling.minReplicas=0 \ - --set cluster.llap.autoscaling.cooldownSeconds=900 \ - --set cluster.llap.autoscaling.gracePeriodSeconds=600 \ - --set cluster.tezAm.autoscaling.enabled=true \ - --set cluster.tezAm.autoscaling.minReplicas=0 \ - --set cluster.tezAm.autoscaling.scaleUpThreshold=5 \ - --set cluster.tezAm.autoscaling.cooldownSeconds=600 \ - --set cluster.tezAm.autoscaling.gracePeriodSeconds=120 + --set cluster.tezAm.autoscaling.enabled=true ``` -**Values file:** +**Values file (for customizing beyond defaults):** ```yaml -# values-autoscaling.yaml +# values-autoscaling.yaml — only override what you need cluster: database: type: postgres @@ -730,48 +734,38 @@ cluster: value: "ozone" hiveServer2: - replicas: 10 # Acts as max replicas when autoscaling is enabled - resources: - requestsCpu: "1" # Required for CPU-based autoscaling trigger - requestsMemory: "2Gi" + replicas: 10 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - minReplicas: 0 # Scale to zero when idle - scaleUpThreshold: 80 # Requests/sec that triggers additional pods - cooldownSeconds: 600 # 10 min before scaling back to 0 - gracePeriodSeconds: 300 + # minReplicas: 0 # default — scale to zero when idle (requires KEDA HTTP Add-on) + # scaleUpThreshold: 80 # default — avg open sessions per pod triggering scale-up + # cooldownSeconds: 300 # default — 5 min before scaling down metastore: - replicas: 6 # Acts as max replicas when autoscaling is enabled - resources: - requestsCpu: "500m" # Required for CPU-based autoscaling trigger - requestsMemory: "1Gi" + replicas: 6 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - minReplicas: 1 # HMS must always be available - cooldownSeconds: 300 - gracePeriodSeconds: 60 + # minReplicas: 0 # default — scale to zero when no connections + # scaleUpThreshold: 75 # default — total open connections triggering scale-up + # cooldownSeconds: 300 # default — 5 min cooldown + # gracePeriodSeconds: 60 # default — fast drain (HMS is stateless) llap: - replicas: 8 # Acts as max replicas when autoscaling is enabled + replicas: 8 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - minReplicas: 0 # Scale to zero when no queries need LLAP - cooldownSeconds: 900 # 15 min — scaling down destroys in-memory cache - gracePeriodSeconds: 600 + # minReplicas: 0 # default — scale to zero when no HS2 sessions + # scaleUpThreshold: 1 # default — total busy slots (queued+running) triggering scale-up + # cooldownSeconds: 900 # default — 15 min (scaling down destroys in-memory cache) + # gracePeriodSeconds: 600 # default — 10 min drain for in-flight fragments tezAm: - replicas: 10 # Acts as max replicas when autoscaling is enabled - resources: - requestsCpu: "500m" # Required for CPU-based autoscaling trigger - requestsMemory: "1Gi" + replicas: 10 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - minReplicas: 0 # Scale to zero when no queries running - scaleUpThreshold: 60 # CPU% when resources set; pending tasks per AM otherwise - scaleDownThreshold: 10 - cooldownSeconds: 600 - gracePeriodSeconds: 120 + # minReplicas: 0 # default — scale to zero when no HS2 sessions + # scaleUpThreshold: 5 # default — CPU% (with resources) or pending tasks (without) + # gracePeriodSeconds: 120 # default — 2 min drain for DAG completion ``` ```bash @@ -790,12 +784,12 @@ When autoscaling is enabled, the operator automatically: **Exported Prometheus Metrics (per component):** -| Component | Metrics | Purpose | +| Component | Key Metrics | Purpose | |-----------|---------|---------| -| **HiveServer2** | `hs2_open_sessions`, `hs2_active_sessions`, `hs2_active_calls_*`, `tez_session_pending_tasks`, `tez_session_running_tasks`, `tez_session_task_backlog_ratio` | Session/query load, Tez AM demand | -| **Metastore** | `api_*_total`, `hive_metastore_open_connections` | API call rates, connection count | -| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_*` | Executor queue depth, daemon health | -| **Tez AM** | `tez_am_*` | DAG execution metrics | +| **HiveServer2** | `hs2_open_sessions`, `hs2_active_sessions`, `tez_session_pending_tasks` | Session count (scaling trigger), Tez AM demand | +| **Metastore** | `hive_metastore_open_connections`, `api_*_total` | Connection count (scaling trigger), API call rates | +| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available (scaling trigger) | +| **Tez AM** | Standard K8s CPU metrics or `tez_session_pending_tasks` (from HS2) | CPU utilization or pending task count (scaling trigger) | ### CPU-Based Scaling and Resource Requests @@ -804,6 +798,18 @@ and Tez AM. KEDA's CPU trigger uses the `Utilization` metric type, which is defi percentage of the container's CPU request. This means **the container must have a CPU request defined** for the trigger to work. +**How it works:** + +- The CPU trigger scales up when pod CPU utilization exceeds `scaleUpThreshold`% of the CPU request +- The `scaleDownThreshold` configures the **activation threshold** — below this CPU%, the + trigger is completely inactive (doesn't participate in scaling decisions) +- Both the CPU trigger and the Prometheus-based trigger (sessions/connections) are evaluated + independently — if **either** exceeds its threshold, the component scales up (OR logic) +- Scale-down only happens when **both** triggers agree load is low (all below threshold) + +This means a long-running CPU-intensive query will keep the pod scaled even if there's +only one session open. Conversely, many idle sessions will keep it scaled even at low CPU. + If you enable autoscaling without setting `resources` for that component, the operator will omit the CPU trigger and rely solely on the Prometheus-based trigger. For Tez AM specifically, without CPU resources the operator uses `tez_session_pending_tasks` (queued @@ -820,6 +826,7 @@ cluster: requestsMemory: "2Gi" autoscaling: enabled: true + scaleDownThreshold: 30 # CPU trigger inactive below 30% (default) metastore: resources: @@ -834,9 +841,17 @@ cluster: requestsMemory: "1Gi" autoscaling: enabled: true + scaleUpThreshold: 60 # For TezAM with resources, this IS the CPU target % + scaleDownThreshold: 10 # CPU trigger inactive below 10% ``` -> **Note:** LLAP scaling uses only Prometheus triggers (`NumQueuedRequests`) +| Setting | Effect on CPU trigger | +|---------|----------------------| +| `resources.requestsCpu` | **Enables** the CPU trigger (required) | +| `scaleUpThreshold` | CPU target % — scales up when utilization exceeds this (default 80) | +| `scaleDownThreshold` | Activation value — CPU trigger ignored below this % (default 30) | + +> **Note:** LLAP scaling uses only Prometheus triggers (total busy slots) > and does not include a CPU trigger, so LLAP does not require `resources` to > be set for autoscaling to work. @@ -885,6 +900,21 @@ Traffic flow: Client → KEDA HTTP Interceptor → (if 0 pods: scale up, wait) → HS2 Service → HS2 Pod ``` +**Via kubectl exec (no local Hive install needed):** + +The Metastore pod is always running (`minReplicas=1`) and has beeline pre-installed. +Connecting through the interceptor wakes HS2 from zero automatically: + +```bash +kubectl exec -it deploy/hive-metastore -- beeline -u "jdbc:hive2://keda-add-ons-http-interceptor-proxy.keda.svc:8080/;transportMode=http;httpPath=cliservice" +``` + +Or connect directly when HS2 is already running: + +```bash +kubectl exec -it deploy/hive-metastore -- beeline -u "jdbc:hive2://hive-hiveserver2:10001/;transportMode=http;httpPath=cliservice" +``` + **Via port-forward (local development):** ```bash @@ -895,44 +925,30 @@ kubectl port-forward -n keda svc/keda-add-ons-http-interceptor-proxy 8080:8080 beeline -u "jdbc:hive2://localhost:8080/;transportMode=http;httpPath=cliservice" ``` -**Via Ingress (production):** +**Via Ingress:** -Create an Ingress that routes your domain to the KEDA interceptor. The key is the -`upstream-vhost` annotation which rewrites the Host header to the internal service -name so the interceptor can match it — no extra operator configuration needed: +Create an Ingress that routes to the KEDA interceptor. Uses [nip.io](https://nip.io) +wildcard DNS so no `/etc/hosts` editing is needed — `hive.127.0.0.1.nip.io` resolves +to `127.0.0.1` automatically: ```bash -cat <<'EOF' | kubectl apply -f - -apiVersion: networking.k8s.io/v1 -kind: Ingress -metadata: - name: hive-interceptor - namespace: keda - annotations: - # Rewrite Host header to internal service name so KEDA interceptor can route it - nginx.ingress.kubernetes.io/upstream-vhost: "hive-hiveserver2.default.svc.cluster.local" -spec: - ingressClassName: nginx - rules: - - host: hive.example.com - http: - paths: - - path: / - pathType: Prefix - backend: - service: - name: keda-add-ons-http-interceptor-proxy - port: - number: 8080 -EOF +kubectl create ingress hive-interceptor -n keda --class=nginx \ + --rule="hive.127.0.0.1.nip.io/*=keda-add-ons-http-interceptor-proxy:8080" \ + --annotation="nginx.ingress.kubernetes.io/upstream-vhost=hive-hiveserver2.default.svc.cluster.local" ``` +> The `upstream-vhost` annotation rewrites the Host header to the internal service +> name so the KEDA interceptor can match and route the request. + Connect via beeline using the Ingress: ```bash -beeline -u "jdbc:hive2://hive.example.com:80/;transportMode=http;httpPath=cliservice" +beeline -u "jdbc:hive2://hive.127.0.0.1.nip.io:80/;transportMode=http;httpPath=cliservice" ``` +> For production, replace `hive.127.0.0.1.nip.io` with your actual domain +> (e.g., `hive.example.com`) and ensure DNS points to your ingress controller. + **Manual wake (fallback without HTTP Add-on):** ```bash @@ -1055,12 +1071,11 @@ kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hi | Value | Default | Description | |-------|---------|-------------| | `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling for this component | -| `cluster..autoscaling.minReplicas` | `2` | Floor replica count during scale-down | -| `cluster..autoscaling.scaleUpThreshold` | `60-80` | Metric threshold triggering scale-up (CPU% for HS2/HMS/TezAM with resources; pending tasks per AM for TezAM without resources; queue depth for LLAP) | -| `cluster..autoscaling.scaleDownThreshold` | `10-30` | Metric percentage threshold triggering scale-down | -| `cluster..autoscaling.cooldownSeconds` | `300-900` | Minimum seconds between scaling events | -| `cluster..autoscaling.gracePeriodSeconds` | `60-600` | Max time (seconds) to wait for graceful drain | -| `cluster.hiveServer2.autoscaling.scaleToZeroHosts` | `[]` | Hostnames for KEDA HTTP interceptor routing (Ingress domain) | +| `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (HS2 requires KEDA HTTP Add-on) | +| `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (sessions for HS2, connections for HMS, busy slots for LLAP, pending tasks or CPU% for TezAM) | +| `cluster..autoscaling.scaleDownThreshold` | `30` | CPU activation threshold below which the CPU trigger is inactive | +| `cluster..autoscaling.cooldownSeconds` | `300` | Seconds to wait after last scale event before scaling down again | +| `cluster..autoscaling.gracePeriodSeconds` | `60-600` | Max time (seconds) to wait for graceful drain before forced termination | --- @@ -1101,35 +1116,21 @@ helm install hive ./helm/hive-operator -f my-values.yaml ### Remove Everything (including dependencies) ```bash -# 1. Uninstall Hive operator (removes ScaledObjects, pods, services via owner references) -helm uninstall hive -kubectl delete crd hiveclusters.hive.apache.org --ignore-not-found - -# 2. Remove HS2 Ingress (if configured for scale-to-zero wake) -kubectl delete ingress hive-hs2-ingress --ignore-not-found - -# 3. Uninstall autoscaling infrastructure (KEDA, HTTP Add-on, Prometheus) +kubectl delete hivecluster --all -A --wait=false --ignore-not-found +kubectl delete ingress hive-interceptor -n keda --ignore-not-found +helm uninstall hive --ignore-not-found +kubectl delete crd hiveclusters.hive.apache.org --wait=false --ignore-not-found +kubectl delete crd --wait=false --ignore-not-found scaledobjects.keda.sh scaledjobs.keda.sh triggerauthentications.keda.sh clustertriggerauthentications.keda.sh httpscaledobjects.http.keda.sh interceptorroutes.http.keda.sh helm uninstall http-add-on -n keda --ignore-not-found helm uninstall keda -n keda --ignore-not-found helm uninstall prometheus -n monitoring --ignore-not-found - -# 4. Remove KEDA CRDs (not removed by helm uninstall) -kubectl delete crd --ignore-not-found \ - scaledobjects.keda.sh \ - scaledjobs.keda.sh \ - triggerauthentications.keda.sh \ - clustertriggerauthentications.keda.sh \ - httpscaledobjects.http.keda.sh - -# 5. Uninstall storage and infrastructure dependencies -helm uninstall ozone postgres zookeeper --ignore-not-found - -# 6. Clean up PVCs, secrets, and namespaces -kubectl delete pvc data-zookeeper-0 --ignore-not-found -kubectl delete pvc data-postgres-postgresql-0 --ignore-not-found +helm uninstall ozone --ignore-not-found +helm uninstall postgres --ignore-not-found +helm uninstall zookeeper --ignore-not-found +kubectl delete pvc data-zookeeper-0 data-postgres-postgresql-0 --ignore-not-found kubectl delete secret hive-db-secret --ignore-not-found -kubectl delete namespace keda --ignore-not-found -kubectl delete namespace monitoring --ignore-not-found +kubectl delete namespace keda --wait=false --ignore-not-found +kubectl delete namespace monitoring --wait=false --ignore-not-found ``` --- diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml index 791c60e0d813..d3df4a5a7868 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml @@ -60,5 +60,5 @@ rules: verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] # KEDA HTTP Add-on for scale-to-zero (wake-from-zero on HTTP request) - apiGroups: ["http.keda.sh"] - resources: ["httpscaledobjects"] + resources: ["httpscaledobjects", "interceptorroutes"] verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/HiveOperatorMain.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/HiveOperatorMain.java index 55bd3372a40d..d02f08fff038 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/HiveOperatorMain.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/HiveOperatorMain.java @@ -19,7 +19,11 @@ package org.apache.hive.kubernetes.operator; import io.javaoperatorsdk.operator.Operator; +import io.javaoperatorsdk.operator.api.config.ControllerConfiguration; +import io.javaoperatorsdk.operator.api.config.ResolvedControllerConfiguration; +import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.reconciler.HiveClusterReconciler; +import org.apache.hive.kubernetes.operator.reconciler.HiveWorkflowSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +40,16 @@ private HiveOperatorMain() { public static void main(String[] args) { LOG.info("Starting Hive Kubernetes Operator"); Operator operator = new Operator(); - operator.register(new HiveClusterReconciler()); + HiveClusterReconciler reconciler = new HiveClusterReconciler(); + // Get the annotation-derived base config, then inject our programmatic workflow spec. + ControllerConfiguration baseConfig = + operator.getConfigurationService().getConfigurationFor(reconciler); + HiveWorkflowSpec workflowSpec = new HiveWorkflowSpec(); + ((ResolvedControllerConfiguration) baseConfig) + .setWorkflowSpec(workflowSpec); + LOG.info("Registered workflow with {} dependent resource specs", + workflowSpec.getDependentResourceSpecs().size()); + operator.register(reconciler, baseConfig); operator.start(); LOG.info("Hive Kubernetes Operator started successfully"); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HadoopConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HadoopConfigMapDependent.java deleted file mode 100644 index 6c0f9308dbc1..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HadoopConfigMapDependent.java +++ /dev/null @@ -1,67 +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.hive.kubernetes.operator.dependent; - -import java.util.Map; - -import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.api.model.ConfigMapBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; -import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the Hadoop core-site.xml ConfigMap for filesystem configuration. */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=hadoop-config," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class HadoopConfigMapDependent - extends HiveDependentResource { - - public static final String COMPONENT = "hadoop-config"; - - public HadoopConfigMapDependent() { - super(ConfigMap.class); - } - - @Override - protected ConfigMap desired(HiveCluster hiveCluster, - Context context) { - Map props = - HiveConfigBuilder.getHadoopCoreSite(hiveCluster.getSpec()); - - return new ConfigMapBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .endMetadata() - .addToData("core-site.xml", HadoopXmlBuilder.buildXml(props)) - .build(); - } - - /** Returns the ConfigMap resource name for this HiveCluster. */ - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hadoop-config"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java new file mode 100644 index 000000000000..935b47e094cb --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java @@ -0,0 +1,153 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.config.informer.Informer; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; + +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; +import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Unified ConfigMap dependent resource for all Hive component configurations. + * Subclassed per component to define the specific XML data and label selector. + */ +public abstract class HiveConfigMapDependent extends HiveDependentResource { + + private final String component; + private final String suffix; + + protected HiveConfigMapDependent(String component, String suffix) { + super(ConfigMap.class); + this.component = component; + this.suffix = suffix; + } + + @Override + protected String getSecondaryResourceName(HiveCluster primary, Context context) { + return primary.getMetadata().getName() + "-" + suffix; + } + + @Override + protected ConfigMap desired(HiveCluster hiveCluster, Context context) { + ConfigMapBuilder builder = + new ConfigMapBuilder().withNewMetadata().withName(hiveCluster.getMetadata().getName() + "-" + suffix) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, component)).endMetadata(); + addData(builder, hiveCluster); + return builder.build(); + } + + /** + * Subclasses add their specific XML data entries. + */ + protected abstract void addData(ConfigMapBuilder builder, HiveCluster hiveCluster); + + /** + * Hadoop core-site.xml ConfigMap for filesystem configuration. + */ + @KubernetesDependent(informer = @Informer(labelSelector = "app.kubernetes.io/component=hadoop-config," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) + public static class Hadoop extends HiveConfigMapDependent { + public Hadoop() { + super("hadoop-config", "hadoop-config"); + } + + @Override + protected void addData(ConfigMapBuilder builder, HiveCluster hiveCluster) { + builder.addToData("core-site.xml", + HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(hiveCluster.getSpec()))); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hadoop-config"; + } + } + + /** + * Metastore metastore-site.xml ConfigMap. + */ + @KubernetesDependent(informer = @Informer(labelSelector = "app.kubernetes.io/component=metastore," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) + public static class Metastore extends HiveConfigMapDependent { + public Metastore() { + super("metastore", "metastore-config"); + } + + @Override + protected void addData(ConfigMapBuilder builder, HiveCluster hiveCluster) { + builder.addToData("metastore-site.xml", + HadoopXmlBuilder.buildXml(HiveConfigBuilder.getMetastoreSite(hiveCluster.getSpec()))); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-metastore-config"; + } + } + + /** + * HiveServer2 hive-site.xml + tez-site.xml ConfigMap. + */ + @KubernetesDependent(informer = @Informer(labelSelector = "app.kubernetes.io/component=hiveserver2," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) + public static class HiveServer2 extends HiveConfigMapDependent { + public HiveServer2() { + super("hiveserver2", "hiveserver2-config"); + } + + @Override + protected void addData(ConfigMapBuilder builder, HiveCluster hiveCluster) { + HiveClusterSpec spec = hiveCluster.getSpec(); + builder.addToData("hive-site.xml", + HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHiveServer2HiveSite(hiveCluster, spec))); + builder.addToData("tez-site.xml", HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec))); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hiveserver2-config"; + } + } + + /** + * LLAP llap-daemon-site.xml ConfigMap. + */ + @KubernetesDependent(informer = @Informer(labelSelector = "app.kubernetes.io/component=llap," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) + public static class Llap extends HiveConfigMapDependent { + public Llap() { + super("llap", "llap-config"); + } + + @Override + protected void addData(ConfigMapBuilder builder, HiveCluster hiveCluster) { + builder.addToData("llap-daemon-site.xml", + HadoopXmlBuilder.buildXml(HiveConfigBuilder.getLlapDaemonSite(hiveCluster.getSpec()))); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-llap-config"; + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 9b1cb75d6553..7fc1290d5375 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import io.fabric8.kubernetes.api.model.AffinityBuilder; import io.fabric8.kubernetes.api.model.Container; import io.fabric8.kubernetes.api.model.ContainerBuilder; @@ -80,32 +81,41 @@ protected HiveDependentResource(Class resourceType) { super(resourceType); } + /** - * Catches 409 AlreadyExists during resource creation caused by - * informer lag — the resource exists on the API server but - * the informer cache hasn't indexed it yet, so JOSDK calls - * create directly. + * Returns the expected Kubernetes resource name for this dependent. + * Used to disambiguate when multiple dependents share the same resource + * type (e.g., multiple ConfigMap or Service dependents). Subclasses that + * share a resource type MUST override this method. + * + * @throws IllegalStateException if not overridden and disambiguation is needed */ - @Override - protected R handleCreate(R desired, P primary, Context

context) { - try { - return super.handleCreate(desired, primary, context); - } catch (KubernetesClientException e) { - if (e.getCode() == 409) { - LOG.info("Resource {} already exists (informer lag), " - + "will reconcile on next event", - desired.getMetadata().getName()); - return desired; - } - throw e; - } + protected String getSecondaryResourceName(P primary, Context

context) { + throw new IllegalStateException( + getClass().getSimpleName() + " must override getSecondaryResourceName() " + + "when multiple dependents share the same resource type"); } @Override public Optional getSecondaryResource(P primary, Context

context) { return eventSource() - .flatMap(es -> es.getSecondaryResource(primary)); + .flatMap(es -> { + Set resources = es.getSecondaryResources(primary); + if (resources.isEmpty()) { + return Optional.empty(); + } + // Always filter by expected name — even when only one resource + // is in the cache. Without this, a single Deployment (e.g. + // metastore) would be handed to HiveServer2's matcher, causing + // a cross-component update loop. + String expectedName = getSecondaryResourceName(primary, + context); + return resources.stream() + .filter(r -> expectedName.equals( + r.getMetadata().getName())) + .findFirst(); + }); } /** @@ -127,6 +137,171 @@ public Matcher.Result match(R actualResource, R desired, return super.match(actualResource, desired, primary, context); } + /** + * Handles 409 Conflict errors during resource creation caused by informer + * cache lag. When the operator creates a resource but the informer hasn't + * yet received the creation event, the framework may attempt to create it + * again. Kubernetes rejects the duplicate with 409 — this handler absorbs + * that expected race and lets the next reconciliation pick up the resource + * from the updated cache. + */ + @Override + protected R handleCreate(R desired, P primary, Context

context) { + try { + return super.handleCreate(desired, primary, context); + } catch (KubernetesClientException e) { + if (e.getCode() == 409) { + LOG.info("Resource {} already exists (informer lag), " + + "will reconcile on next event", + desired.getMetadata().getName()); + return desired; + } + throw e; + } + } + + /** + * Resolves the replica count to set in the desired workload spec. + * When autoscaling is enabled and the workload already exists, the current + * replica count is preserved (KEDA/HPA manages it). On initial creation + * the provided fallback is used. + * + * @param primary the HiveCluster primary resource + * @param context the reconciliation context + * @param autoscaling autoscaling spec for this component (may be null) + * @param staticReplicas replica count from the spec (used when autoscaling is off) + * @param initialReplicas replica count on first creation when autoscaling is on + */ + @SuppressWarnings("unchecked") + protected Integer resolveReplicaCount(P primary, Context

context, + AutoscalingSpec autoscaling, int staticReplicas, int initialReplicas) { + if (autoscaling == null || !autoscaling.isEnabled()) { + return staticReplicas; + } + return getSecondaryResource(primary, context) + .map(existing -> { + if (existing instanceof io.fabric8.kubernetes.api.model.apps.Deployment d) { + return d.getSpec().getReplicas(); + } else if (existing instanceof io.fabric8.kubernetes.api.model.apps.StatefulSet s) { + return s.getSpec().getReplicas(); + } + return initialReplicas; + }) + .orElse(initialReplicas); + } + + /** + * Builds a preStop drain script that polls a single Prometheus metric + * (from the JMX Exporter at localhost:9404/metrics) until the value + * reaches zero, then exits to allow graceful pod termination. + * + * @param startupMessage logged at the start (e.g. "Waiting for open connections to drain") + * @param metricName Prometheus metric name (used in grep and log messages) + * @param varName shell variable name for the extracted value (e.g. "CONNS") + * @param idleMessage logged when idle condition is met (e.g. "All connections drained. Shutting down.") + * @param sleepSeconds polling interval in seconds + * @param maxRetries max consecutive curl failures before giving up + * @param prefixCommands optional commands to run before the polling loop (may be null) + */ + protected static String buildDrainScript( + String startupMessage, String metricName, String varName, + String idleMessage, int sleepSeconds, int maxRetries, + List prefixCommands) { + List lines = new ArrayList<>(); + lines.add("#!/bin/bash"); + if (prefixCommands != null) { + lines.addAll(prefixCommands); + } + lines.add("echo '[preStop] " + startupMessage + + " (polling localhost:9404/metrics)...'"); + lines.add("RETRIES=0"); + lines.add("while true; do"); + lines.add(" RESPONSE=$(curl -sf http://localhost:9404/metrics)"); + lines.add(" if [ $? -ne 0 ]; then"); + lines.add(" RETRIES=$((RETRIES+1))"); + lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\""); + lines.add(" if [ $RETRIES -ge " + maxRetries + " ]; then"); + lines.add(" echo '[preStop] JMX Exporter not responding after " + + (maxRetries * sleepSeconds) + "s. Proceeding with shutdown.'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" sleep " + sleepSeconds + "; continue"); + lines.add(" fi"); + lines.add(" " + varName + "=$(echo \"$RESPONSE\" | grep '^" + + metricName + " ' | awk '{print $2}')"); + lines.add(" if [ -z \"$" + varName + "\" ]; then"); + lines.add(" echo '[preStop] WARNING: " + metricName + + " metric not found. JMX Exporter may not be configured.'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" if [ \"${" + varName + "%.*}\" -le 0 ] 2>/dev/null; then"); + lines.add(" echo '[preStop] " + idleMessage + "'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" echo \"[preStop] " + metricName + "=$" + varName + " - waiting...\""); + lines.add(" RETRIES=0"); + lines.add(" sleep " + sleepSeconds); + lines.add("done"); + return String.join("\n", lines); + } + + /** + * Builds a preStop drain script that polls two Prometheus metrics and + * waits until available >= total (all executors idle). Used by LLAP. + * + * @param startupMessage logged at the start + * @param metricGrepA grep pattern for the first metric (e.g. includes trailing '{') + * @param varNameA shell variable for the first metric value (e.g. "AVAILABLE") + * @param metricGrepB grep pattern for the second metric + * @param varNameB shell variable for the second metric value (e.g. "TOTAL") + * @param notFoundWarning warning message when metrics are not found + * @param idleMessage logged when idle condition is met + * @param waitingFormat format for waiting log (with shell variable references) + * @param sleepSeconds polling interval in seconds + * @param maxRetries max consecutive curl failures before giving up + */ + protected static String buildDualMetricDrainScript( + String startupMessage, + String metricGrepA, String varNameA, + String metricGrepB, String varNameB, + String notFoundWarning, String idleMessage, + String waitingFormat, int sleepSeconds, int maxRetries) { + List lines = new ArrayList<>(); + lines.add("#!/bin/bash"); + lines.add("echo '[preStop] " + startupMessage + + " (polling localhost:9404/metrics)...'"); + lines.add("RETRIES=0"); + lines.add("while true; do"); + lines.add(" RESPONSE=$(curl -sf http://localhost:9404/metrics)"); + lines.add(" if [ $? -ne 0 ]; then"); + lines.add(" RETRIES=$((RETRIES+1))"); + lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\""); + lines.add(" if [ $RETRIES -ge " + maxRetries + " ]; then"); + lines.add(" echo '[preStop] JMX Exporter not responding after " + + (maxRetries * sleepSeconds) + "s. Proceeding with shutdown.'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" sleep " + sleepSeconds + "; continue"); + lines.add(" fi"); + lines.add(" " + varNameA + "=$(echo \"$RESPONSE\" | grep '^" + + metricGrepA + "' | awk '{print $2}')"); + lines.add(" " + varNameB + "=$(echo \"$RESPONSE\" | grep '^" + + metricGrepB + "' | awk '{print $2}')"); + lines.add(" if [ -z \"$" + varNameA + "\" ] || [ -z \"$" + varNameB + "\" ]; then"); + lines.add(" echo '[preStop] WARNING: " + notFoundWarning + "'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" if [ \"${" + varNameA + "%.*}\" -ge \"${" + varNameB + "%.*}\" ] 2>/dev/null; then"); + lines.add(" echo '[preStop] " + idleMessage + "'"); + lines.add(" break"); + lines.add(" fi"); + lines.add(" echo \"[preStop] " + waitingFormat + "\""); + lines.add(" RETRIES=0"); + lines.add(" sleep " + sleepSeconds); + lines.add("done"); + return String.join("\n", lines); + } + /** * Computes a SHA-256 hash of the given input strings. * Used to annotate pod templates so that config changes trigger rolling updates. @@ -237,8 +412,8 @@ protected static void buildMetastoreVolumes( .withMountPath(CONF_MOUNT_PATH).build()); volumes.add(buildProjectedConfigVolume("hive-config", - MetastoreConfigMapDependent.resourceName(hiveCluster), - HadoopConfigMapDependent.resourceName(hiveCluster))); + HiveConfigMapDependent.Metastore.resourceName(hiveCluster), + HiveConfigMapDependent.Hadoop.resourceName(hiveCluster))); } /** Builds Kubernetes ResourceRequirements from the operator's spec. */ @@ -424,6 +599,65 @@ protected static Probe buildTcpProbe(int port, ProbeSpec spec, int defaultInitia return builder.build(); } + /** + * Applies the autoscaling lifecycle to a workload's pod template: sets a preStop + * exec lifecycle hook, terminationGracePeriodSeconds, and Prometheus scrape annotations. + * + * @param podSpec the pod spec of the workload (Deployment or StatefulSet) + * @param podMetadata the pod template metadata (for annotations) + * @param preStopScript the shell script to run in the preStop hook + * @param gracePeriodSeconds termination grace period + */ + protected static void applyAutoscalingLifecycle( + io.fabric8.kubernetes.api.model.PodSpec podSpec, + io.fabric8.kubernetes.api.model.ObjectMeta podMetadata, + String preStopScript, int gracePeriodSeconds) { + io.fabric8.kubernetes.api.model.Lifecycle lifecycle = + new io.fabric8.kubernetes.api.model.LifecycleBuilder() + .withNewPreStop() + .withNewExec() + .withCommand("/bin/bash", "-c", preStopScript) + .endExec() + .endPreStop() + .build(); + podSpec.getContainers().get(0).setLifecycle(lifecycle); + podSpec.setTerminationGracePeriodSeconds((long) gracePeriodSeconds); + podMetadata.getAnnotations().put("prometheus.io/scrape", "true"); + podMetadata.getAnnotations().put("prometheus.io/port", + String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); + podMetadata.getAnnotations().put("prometheus.io/path", "/metrics"); + } + + /** + * Appends user-provided volumes and volume mounts to a workload's pod template. + * Handles both global (spec-level) and component-specific extras. + * + * @param podSpec the pod spec + * @param globalVolumes spec.volumes() (may be null) + * @param globalVolumeMounts spec.volumeMounts() (may be null) + * @param extraVolumes component-specific extraVolumes (may be null) + * @param extraVolumeMounts component-specific extraVolumeMounts (may be null) + */ + protected static void appendUserVolumes( + io.fabric8.kubernetes.api.model.PodSpec podSpec, + List globalVolumes, + List globalVolumeMounts, + List extraVolumes, + List extraVolumeMounts) { + if (globalVolumes != null) { + podSpec.getVolumes().addAll(globalVolumes); + } + if (globalVolumeMounts != null) { + podSpec.getContainers().get(0).getVolumeMounts().addAll(globalVolumeMounts); + } + if (extraVolumes != null) { + podSpec.getVolumes().addAll(extraVolumes); + } + if (extraVolumeMounts != null) { + podSpec.getContainers().get(0).getVolumeMounts().addAll(extraVolumeMounts); + } + } + /** Path where the JMX Exporter agent JAR is stored inside the pod. */ protected static final String JMX_EXPORTER_DIR = "/opt/jmx-exporter"; protected static final String JMX_EXPORTER_JAR = JMX_EXPORTER_DIR + "/jmx_prometheus_javaagent.jar"; @@ -485,7 +719,8 @@ protected static void addJmxExporter( // Expose the metrics port ports.add(new io.fabric8.kubernetes.api.model.ContainerPortBuilder() .withName("metrics") - .withContainerPort(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT).build()); + .withContainerPort(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT) + .withProtocol("TCP").build()); // Add javaagent flag to the appropriate JVM opts env var. // LLAP uses LLAP_DAEMON_OPTS (its startup script ignores SERVICE_OPTS). @@ -536,7 +771,7 @@ private static String buildJmxExporterConfig(String component) { sb.append("- pattern: 'metrics<>Count'\n"); sb.append(" name: api_$1_total\n"); sb.append(" type: COUNTER\n"); - sb.append("- pattern: 'metrics<>Value'\n"); + sb.append("- pattern: 'metrics<>Count'\n"); sb.append(" name: hive_metastore_open_connections\n"); sb.append(" type: GAUGE\n"); break; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java index feff8775a6f4..6db1482f3bd1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java @@ -18,11 +18,16 @@ package org.apache.hive.kubernetes.operator.dependent; +import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.javaoperatorsdk.operator.api.config.informer.InformerEventSourceConfiguration; import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.EventSourceContext; import io.javaoperatorsdk.operator.api.reconciler.dependent.GarbageCollected; import io.javaoperatorsdk.operator.processing.GroupVersionKind; import io.javaoperatorsdk.operator.processing.dependent.Creator; @@ -52,6 +57,22 @@ protected HiveGenericDependentResource(GroupVersionKind gvk) { super(gvk); } + /** + * Adds a generation-aware update filter so that KEDA/controller status + * patches (which don't increment metadata.generation) do not trigger + * unnecessary reconciliation loops. + */ + @Override + protected InformerEventSourceConfiguration.Builder + informerConfigurationBuilder(EventSourceContext context) { + return super.informerConfigurationBuilder(context) + .withOnUpdateFilter((newResource, oldResource) -> { + Long newGen = newResource.getMetadata().getGeneration(); + Long oldGen = oldResource.getMetadata().getGeneration(); + return !Objects.equals(newGen, oldGen); + }); + } + /** * Returns the expected Kubernetes resource name for this dependent given the primary. * Used to discriminate between multiple secondary resources of the same GVK @@ -70,4 +91,71 @@ public Optional getSecondaryResource( .filter(r -> expectedName.equals(r.getMetadata().getName())) .findFirst(); } + + /** + * Builds the nested "advanced" HPA behavior configuration for a KEDA ScaledObject. + * + * @param scaleDownStabilization stabilizationWindowSeconds for scale-down + * @param scaleDownPolicyType policy type (e.g. "Pods", "Percent") + * @param scaleDownValue policy value + * @param scaleDownPeriod policy periodSeconds + * @param scaleUpStabilization stabilizationWindowSeconds for scale-up + * @param scaleUpPolicyType policy type (e.g. "Pods", "Percent") + * @param scaleUpValue policy value + * @param scaleUpPeriod policy periodSeconds + */ + protected static Map buildHpaBehavior( + int scaleDownStabilization, String scaleDownPolicyType, + int scaleDownValue, int scaleDownPeriod, + int scaleUpStabilization, String scaleUpPolicyType, + int scaleUpValue, int scaleUpPeriod) { + return Map.of( + "horizontalPodAutoscalerConfig", Map.of( + "behavior", Map.of( + "scaleDown", Map.of( + "stabilizationWindowSeconds", scaleDownStabilization, + "policies", List.of(Map.of( + "type", scaleDownPolicyType, + "value", scaleDownValue, + "periodSeconds", scaleDownPeriod + )) + ), + "scaleUp", Map.of( + "stabilizationWindowSeconds", scaleUpStabilization, + "policies", List.of(Map.of( + "type", scaleUpPolicyType, + "value", scaleUpValue, + "periodSeconds", scaleUpPeriod + )) + ) + ) + ) + ); + } + + /** + * Builds the HS2 cross-component activation trigger used by LLAP and TezAM. + * Uses {@code (max(hs2_open_sessions{...}) > bool 0) or vector(0)} so the + * result is always 0 or 1, preventing zombie sessions from driving proportional scaling. + * Threshold is set to maxReplicas so desired = ceil(1/max) = 1 (activation only). + * + * @param namespace the Kubernetes namespace + * @param hs2TargetName the HS2 deployment name (for pod label matching) + * @param maxReplicas the max replicas of the component (used as threshold) + */ + protected static Map buildHs2ActivationTrigger( + String namespace, String hs2TargetName, int maxReplicas) { + return Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_open_sessions_activation", + "query", String.format( + "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(maxReplicas), + "activationThreshold", "0" + ) + ); + } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java new file mode 100644 index 000000000000..2942a5b674bf --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -0,0 +1,103 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; +import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.config.informer.Informer; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Unified PodDisruptionBudget dependent resource for all Hive components. + * Ensures at least one pod remains available during voluntary disruptions + * (scale-down, node drain, rolling updates). + *

+ * Subclassed per component (HS2, Metastore, LLAP, TezAM) only to satisfy + * JOSDK's requirement for distinct no-arg-constructible classes in the workflow. + */ +public abstract class HivePdbDependent + extends HiveDependentResource { + + private final String component; + + protected HivePdbDependent(String component) { + super(PodDisruptionBudget.class); + this.component = component; + } + + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return primary.getMetadata().getName() + "-" + component + "-pdb"; + } + + @Override + protected PodDisruptionBudget desired(HiveCluster hiveCluster, + Context context) { + return new PodDisruptionBudgetBuilder() + .withNewMetadata() + .withName(hiveCluster.getMetadata().getName() + "-" + component + "-pdb") + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, component)) + .endMetadata() + .withNewSpec() + .withMinAvailable(new IntOrString(1)) + .withNewSelector() + .withMatchLabels(Labels.selectorForComponent(hiveCluster, component)) + .endSelector() + .endSpec() + .build(); + } + + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=hiveserver2," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class HiveServer2 extends HivePdbDependent { + public HiveServer2() { super("hiveserver2"); } + } + + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=metastore," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class Metastore extends HivePdbDependent { + public Metastore() { super("metastore"); } + } + + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=llap," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class Llap extends HivePdbDependent { + public Llap() { super("llap"); } + } + + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=tezam," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class TezAm extends HivePdbDependent { + public TezAm() { super("tezam"); } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java new file mode 100644 index 000000000000..f5efb1302bd3 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java @@ -0,0 +1,360 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Unified KEDA ScaledObject dependent resource for metric-based autoscaling. + * Subclassed per component to define component-specific triggers, HPA behavior, + * and target workload kind. + *

+ * Note: When HS2 minReplicas is 0, the ScaledObject includes an external-push + * trigger from the KEDA HTTP Add-on (via InterceptorRoute) for wake-from-zero. + */ +public abstract class HiveScaledObjectDependent extends HiveGenericDependentResource { + + private final String component; + private final String targetKind; + + protected HiveScaledObjectDependent(String component, String targetKind) { + super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); + this.component = component; + this.targetKind = targetKind; + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = getAutoscalingSpec(hiveCluster); + int maxReplicas = getMaxReplicas(hiveCluster); + String targetName = hiveCluster.getMetadata().getName() + "-" + component; + + Map spec = new HashMap<>(); + spec.put("scaleTargetRef", Map.of( + "apiVersion", "apps/v1", + "kind", targetKind, + "name", targetName + )); + int minReplicaCount = Math.max(1, autoscaling.minReplicas()); + spec.put("minReplicaCount", minReplicaCount); + spec.put("maxReplicaCount", maxReplicas); + if (autoscaling.minReplicas() == 0) { + spec.put("idleReplicaCount", 0); + } + spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); + spec.put("pollingInterval", getPollingInterval()); + spec.put("advanced", getAdvanced(hiveCluster, autoscaling, maxReplicas)); + spec.put("triggers", getTriggers(hiveCluster, autoscaling, maxReplicas, targetName)); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("keda.sh/v1alpha1") + .withKind("ScaledObject") + .withNewMetadata() + .withName(targetName + "-scaledobject") + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, component)) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-" + component + "-scaledobject"; + } + + /** Returns the autoscaling spec for the component. */ + protected abstract AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster); + + /** Returns max replicas (typically the static replicas count from spec). */ + protected abstract int getMaxReplicas(HiveCluster hiveCluster); + + /** Returns the KEDA polling interval in seconds. */ + protected abstract int getPollingInterval(); + + /** Returns the "advanced" section (HPA behavior configuration). */ + protected abstract Map getAdvanced( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas); + + /** Returns the list of KEDA triggers. */ + protected abstract List> getTriggers( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, + int maxReplicas, String targetName); + + /** + * HiveServer2 ScaledObject: scales on hs2_active_sessions + CPU. + */ + public static class HiveServer2 extends HiveScaledObjectDependent { + public HiveServer2() { super("hiveserver2", "Deployment"); } + + @Override + protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { + return hiveCluster.getSpec().hiveServer2().autoscaling(); + } + + @Override + protected int getMaxReplicas(HiveCluster hiveCluster) { + return hiveCluster.getSpec().hiveServer2().replicas(); + } + + @Override + protected int getPollingInterval() { return 30; } + + @Override + protected Map getAdvanced( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + return buildHpaBehavior( + autoscaling.cooldownSeconds(), "Pods", 1, 60, + 60, "Percent", 100, 60); + } + + @Override + protected List> getTriggers( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, + int maxReplicas, String targetName) { + List> triggers = new ArrayList<>(); + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hs2_open_sessions", + "query", String.format( + "avg(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + )); + if (hiveCluster.getSpec().hiveServer2().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", String.valueOf(autoscaling.scaleUpThreshold()), + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + } + // When scale-to-zero is enabled, add KEDA HTTP Add-on external-push + // trigger to wake HS2 from 0 when requests arrive at the interceptor. + if (autoscaling.minReplicas() == 0) { + String routeName = HiveServer2InterceptorRouteDependent.resourceName(hiveCluster); + triggers.add(Map.of( + "type", "external-push", + "metadata", Map.of( + "scalerAddress", + "keda-add-ons-http-external-scaler.keda:9090", + "interceptorRoute", routeName + ) + )); + } + return triggers; + } + } + + /** + * Metastore ScaledObject: scales on open_connections + CPU. + */ + public static class Metastore extends HiveScaledObjectDependent { + public Metastore() { super("metastore", "Deployment"); } + + @Override + protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { + return hiveCluster.getSpec().metastore().autoscaling(); + } + + @Override + protected int getMaxReplicas(HiveCluster hiveCluster) { + return hiveCluster.getSpec().metastore().replicas(); + } + + @Override + protected int getPollingInterval() { return 30; } + + @Override + protected Map getAdvanced( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + return buildHpaBehavior( + autoscaling.cooldownSeconds(), "Pods", 1, 60, + 120, "Percent", 50, 60); + } + + @Override + protected List> getTriggers( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, + int maxReplicas, String targetName) { + List> triggers = new ArrayList<>(); + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "hive_metastore_open_connections", + "query", String.format( + "sum(hive_metastore_open_connections{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + )); + if (hiveCluster.getSpec().metastore().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", String.valueOf(autoscaling.scaleUpThreshold()), + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + } + return triggers; + } + } + + /** + * LLAP ScaledObject: scales on NumQueuedRequests + HS2 activation trigger. + * Scale-down is slow (preserves in-memory cache). + */ + public static class Llap extends HiveScaledObjectDependent { + public Llap() { super("llap", "StatefulSet"); } + + @Override + protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { + return hiveCluster.getSpec().llap().autoscaling(); + } + + @Override + protected int getMaxReplicas(HiveCluster hiveCluster) { + return hiveCluster.getSpec().llap().replicas(); + } + + @Override + protected int getPollingInterval() { return 5; } + + @Override + protected Map getAdvanced( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + return buildHpaBehavior( + autoscaling.cooldownSeconds(), "Pods", 1, autoscaling.cooldownSeconds(), + 0, "Pods", maxReplicas, 15); + } + + @Override + protected List> getTriggers( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, + int maxReplicas, String targetName) { + String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; + String namespace = hiveCluster.getMetadata().getNamespace(); + return List.of( + Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "llap_total_busy_slots", + "query", String.format( + "avg(" + + "hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + " + on(pod) hadoop_llapdaemon_executornumexecutorsconfigured{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + " - on(pod) hadoop_llapdaemon_executornumexecutorsavailable{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + ") or vector(0)", + namespace, targetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + ), + buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas) + ); + } + } + + /** + * TezAM ScaledObject: scales on CPU (or pending tasks) + HS2 activation trigger. + * Tez AMs run in a warm pool; claimed AMs consume CPU, idle ones do not. + */ + public static class TezAm extends HiveScaledObjectDependent { + public TezAm() { super("tezam", "StatefulSet"); } + + @Override + protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { + return hiveCluster.getSpec().tezAm().autoscaling(); + } + + @Override + protected int getMaxReplicas(HiveCluster hiveCluster) { + return hiveCluster.getSpec().tezAm().replicas(); + } + + @Override + protected int getPollingInterval() { return 5; } + + @Override + protected Map getAdvanced( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + return buildHpaBehavior( + autoscaling.cooldownSeconds(), "Pods", 1, 60, + 60, "Pods", 2, 30); + } + + @Override + protected List> getTriggers( + HiveCluster hiveCluster, AutoscalingSpec autoscaling, + int maxReplicas, String targetName) { + String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; + String namespace = hiveCluster.getMetadata().getNamespace(); + List> triggers = new ArrayList<>(); + if (hiveCluster.getSpec().tezAm().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "Utilization", + "metadata", Map.of( + "value", String.valueOf(autoscaling.scaleUpThreshold()), + "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) + ) + )); + triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); + } else { + triggers.add(Map.of( + "type", "prometheus", + "metadata", Map.of( + "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", + "metricName", "tez_session_pending_tasks", + "query", String.format( + "sum(tez_session_pending_tasks{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + namespace, hs2TargetName), + "threshold", String.valueOf(autoscaling.scaleUpThreshold()), + "activationThreshold", "0" + ) + )); + triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); + } + return triggers; + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ConfigMapDependent.java deleted file mode 100644 index 9bb0597cc960..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ConfigMapDependent.java +++ /dev/null @@ -1,72 +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.hive.kubernetes.operator.dependent; - -import java.util.Map; - -import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.api.model.ConfigMapBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; -import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; -import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the hive-site.xml ConfigMap for HiveServer2. */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=hiveserver2," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class HiveServer2ConfigMapDependent - extends HiveDependentResource { - - public static final String COMPONENT = "hiveserver2"; - - public HiveServer2ConfigMapDependent() { - super(ConfigMap.class); - } - - @Override - protected ConfigMap desired(HiveCluster hiveCluster, - Context context) { - HiveClusterSpec spec = hiveCluster.getSpec(); - - Map props = - HiveConfigBuilder.getHiveServer2HiveSite(hiveCluster, spec); - Map tezProps = HiveConfigBuilder.getTezSite(spec); - - return new ConfigMapBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .endMetadata() - .addToData("hive-site.xml", HadoopXmlBuilder.buildXml(props)) - .addToData("tez-site.xml", HadoopXmlBuilder.buildXml(tezProps)) - .build(); - } - - /** Returns the ConfigMap resource name for this HiveCluster. */ - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hiveserver2-config"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index c61383ac0f5a..c08796c218a7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -26,8 +26,6 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.Lifecycle; -import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder; @@ -58,6 +56,12 @@ public HiveServer2DeploymentDependent() { super(Deployment.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -139,13 +143,13 @@ protected Deployment desired(HiveCluster hiveCluster, List ports = new ArrayList<>(); ports.add(new ContainerPortBuilder() .withName("thrift") - .withContainerPort(hs2ThriftPort).build()); + .withContainerPort(hs2ThriftPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() .withName("http") - .withContainerPort(hs2HttpPort).build()); + .withContainerPort(hs2HttpPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() .withName("webui") - .withContainerPort(hs2WebUiPort).build()); + .withContainerPort(hs2WebUiPort).withProtocol("TCP").build()); // Probes target the HTTP transport port (default mode) Probe readinessProbe = buildTcpProbe(hs2HttpPort, hs2.readinessProbe(), 15, 10, 3); @@ -165,8 +169,8 @@ protected Deployment desired(HiveCluster hiveCluster, List volumes = new ArrayList<>(); volumes.add(buildProjectedConfigVolume("hive-config", - HiveServer2ConfigMapDependent.resourceName(hiveCluster), - HadoopConfigMapDependent.resourceName(hiveCluster))); + HiveConfigMapDependent.HiveServer2.resourceName(hiveCluster), + HiveConfigMapDependent.Hadoop.resourceName(hiveCluster))); if (tezAmEnabled) { volumeMounts.add( @@ -211,18 +215,12 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled and the Deployment already exists, preserve the current - // replica count (managed by KEDA/HPA). On initial creation: - // - minReplicas == 0 (scale-to-zero): start at 0, KEDA HTTPScaledObject handles wake-up - // - minReplicas > 0: start at configured replicas - boolean autoscalingEnabled = hs2.autoscaling() != null && hs2.autoscaling().isEnabled(); - Integer replicas = hs2.replicas(); - if (autoscalingEnabled) { - int initialReplicas = hs2.autoscaling().minReplicas() == 0 ? 0 : hs2.replicas(); - replicas = getSecondaryResource(hiveCluster, context) - .map(d -> d.getSpec().getReplicas()) - .orElse(initialReplicas); - } + // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). + AutoscalingSpec hs2Autoscaling = hs2.autoscaling(); + int initialReplicas = hs2Autoscaling != null && hs2Autoscaling.minReplicas() == 0 + ? 0 : hs2.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, hs2Autoscaling, hs2.replicas(), initialReplicas); Deployment deployment = new DeploymentBuilder() .withNewMetadata() @@ -264,71 +262,24 @@ protected Deployment desired(HiveCluster hiveCluster, deployment.getSpec().getTemplate().getSpec(), selectorLabels); // Graceful scale-down: deregister from ZK, then poll JMX Exporter (port 9404) for sessions. - // Uses flat Prometheus text format — same metric KEDA reads — not brittle JSON parsing. if (autoscaling.isEnabled()) { - String preStopScript = String.join("\n", - "#!/bin/bash", + List zkDeregister = List.of( "echo '[preStop] Deregistering HiveServer2 from ZooKeeper...'", - "hive --service hiveserver2 --deregister || echo '[preStop] WARNING: ZK deregister failed'", - "echo '[preStop] Waiting for open sessions to drain (polling localhost:9404/metrics)...'", - "RETRIES=0", - "while true; do", - " RESPONSE=$(curl -sf http://localhost:9404/metrics)", - " if [ $? -ne 0 ]; then", - " RETRIES=$((RETRIES+1))", - " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", - " if [ $RETRIES -ge 6 ]; then", - " echo '[preStop] JMX Exporter not responding after 30s. Proceeding with shutdown.'", - " break", - " fi", - " sleep 5; continue", - " fi", - " SESSIONS=$(echo \"$RESPONSE\" | grep '^hs2_open_sessions ' | awk '{print $2}')", - " if [ -z \"$SESSIONS\" ]; then", - " echo '[preStop] WARNING: hs2_open_sessions metric not found. JMX Exporter may not be configured.'", - " break", - " fi", - " if [ \"${SESSIONS%.*}\" -le 0 ] 2>/dev/null; then", - " echo '[preStop] All sessions drained. Shutting down.'", - " break", - " fi", - " echo \"[preStop] hs2_open_sessions=$SESSIONS — waiting...\"", - " RETRIES=0", - " sleep 5", - "done"); - Lifecycle lifecycle = new LifecycleBuilder() - .withNewPreStop() - .withNewExec() - .withCommand("/bin/bash", "-c", preStopScript) - .endExec() - .endPreStop() - .build(); - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); - deployment.getSpec().getTemplate().getSpec() - .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); - // Prometheus scrape annotations for JMX Exporter metrics endpoint - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/scrape", "true"); - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/path", "/metrics"); + "hive --service hiveserver2 --deregister || echo '[preStop] WARNING: ZK deregister failed'"); + String preStopScript = buildDrainScript( + "Waiting for open sessions to drain", + "hs2_open_sessions", "SESSIONS", + "All sessions drained. Shutting down.", + 5, 6, zkDeregister); + applyAutoscalingLifecycle( + deployment.getSpec().getTemplate().getSpec(), + deployment.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds()); } - if (spec.volumes() != null) { - deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); - } - if (spec.volumeMounts() != null) { - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(spec.volumeMounts()); - } - if (hs2.extraVolumes() != null) { - deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(hs2.extraVolumes()); - } - if (hs2.extraVolumeMounts() != null) { - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(hs2.extraVolumeMounts()); - } + appendUserVolumes(deployment.getSpec().getTemplate().getSpec(), + spec.volumes(), spec.volumeMounts(), + hs2.extraVolumes(), hs2.extraVolumeMounts()); return deployment; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java index 74794ad8e16a..055bd878d2f3 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java @@ -70,11 +70,12 @@ protected GenericKubernetesResource desired(HiveCluster hiveCluster, Map spec = new HashMap<>(); // Hosts the interceptor matches for routing. - // Uses internal service DNS names (Ingress rewrites Host header to match these) - // plus localhost for kubectl port-forward scenarios. + // Includes: internal service FQDN, short name, interceptor proxy name + // (for in-cluster kubectl exec), and localhost (for port-forward). spec.put("hosts", List.of( serviceName + "." + namespace + ".svc.cluster.local", serviceName, + "keda-add-ons-http-interceptor-proxy.keda.svc", "localhost" )); spec.put("pathPrefixes", List.of("/")); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java new file mode 100644 index 000000000000..de6e3bb71d5c --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java @@ -0,0 +1,116 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.fabric8.kubernetes.api.model.GenericKubernetesResource; +import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.processing.GroupVersionKind; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Manages a KEDA InterceptorRoute for HiveServer2 scale-to-zero routing. + *

+ * Unlike HTTPScaledObject, InterceptorRoute only configures interceptor + * routing without auto-creating a ScaledObject. This allows us to manage + * scaling entirely through a single Prometheus-based ScaledObject that + * combines session/CPU awareness with the HTTP interceptor wake-from-zero + * trigger. + *

+ * Requires the KEDA HTTP Add-on to be installed in the cluster. + */ +public class HiveServer2InterceptorRouteDependent extends HiveGenericDependentResource { + + public HiveServer2InterceptorRouteDependent() { + super(new GroupVersionKind("http.keda.sh", "v1beta1", "InterceptorRoute")); + } + + @Override + protected GenericKubernetesResource desired(HiveCluster hiveCluster, + Context context) { + AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); + String clusterName = hiveCluster.getMetadata().getName(); + String namespace = hiveCluster.getMetadata().getNamespace(); + String serviceName = clusterName + "-hiveserver2"; + + int httpPort = ConfigUtils.getInt( + hiveCluster.getSpec().hiveServer2().configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); + + // Hosts the interceptor matches for routing + List hosts = new ArrayList<>(List.of( + serviceName + "." + namespace + ".svc.cluster.local", + serviceName, + "keda-add-ons-http-interceptor-proxy.keda.svc", + "localhost" + )); + + Map spec = new HashMap<>(); + + // Target backend service + spec.put("target", Map.of( + "service", serviceName, + "port", httpPort + )); + + // Routing rules + spec.put("rules", List.of( + Map.of( + "hosts", hosts, + "paths", List.of(Map.of("value", "/")) + ) + )); + + // Scaling metric (required field, used by interceptor for queue management) + spec.put("scalingMetric", Map.of( + "concurrency", Map.of( + "targetValue", autoscaling.scaleUpThreshold() + ) + )); + + return new GenericKubernetesResourceBuilder() + .withApiVersion("http.keda.sh/v1beta1") + .withKind("InterceptorRoute") + .withNewMetadata() + .withName(resourceName(hiveCluster)) + .withNamespace(namespace) + .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) + .endMetadata() + .withAdditionalProperties(Map.of("spec", spec)) + .build(); + } + + @Override + protected String getResourceName(HiveCluster hiveCluster) { + return resourceName(hiveCluster); + } + + public static String resourceName(HiveCluster hiveCluster) { + return hiveCluster.getMetadata().getName() + "-hiveserver2-route"; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java deleted file mode 100644 index 054881f9644d..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2PdbDependent.java +++ /dev/null @@ -1,62 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * PodDisruptionBudget for HiveServer2. - * Ensures at least one HS2 pod remains available during voluntary disruptions - * (scale-down, node drain, rolling updates) to prevent query failures. - */ -public class HiveServer2PdbDependent - extends CRUDKubernetesDependentResource { - - public HiveServer2PdbDependent() { - super(PodDisruptionBudget.class); - } - - @Override - protected PodDisruptionBudget desired(HiveCluster hiveCluster, - Context context) { - return new PodDisruptionBudgetBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) - .endMetadata() - .withNewSpec() - .withMinAvailable(new IntOrString(1)) - .withNewSelector() - .withMatchLabels(Labels.selectorForComponent(hiveCluster, "hiveserver2")) - .endSelector() - .endSpec() - .build(); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hiveserver2-pdb"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java deleted file mode 100644 index 196d97ce8b8f..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ScaledObjectDependent.java +++ /dev/null @@ -1,149 +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.hive.kubernetes.operator.dependent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA ScaledObject for HiveServer2 autoscaling. - *

- * Scale-up triggers (OR): - * - hs2_active_sessions > scaleUpThreshold% of hive.server2.session.max (1 min) - * - Pod CPU > 75% - *

- * Scale-down triggers (AND): - * - hs2_open_sessions < scaleDownThreshold% of max - * - CPU < 30% - *

- * Cooldown: configurable (default 600s / 10 minutes) - */ -public class HiveServer2ScaledObjectDependent extends HiveGenericDependentResource { - - public HiveServer2ScaledObjectDependent() { - super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); - int maxReplicas = hiveCluster.getSpec().hiveServer2().replicas(); - String targetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; - - Map spec = new HashMap<>(); - spec.put("scaleTargetRef", Map.of( - "apiVersion", "apps/v1", - "kind", "Deployment", - "name", targetName - )); - // KEDA requires idleReplicaCount < minReplicaCount. - // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). - int minReplicaCount = Math.max(1, autoscaling.minReplicas()); - spec.put("minReplicaCount", minReplicaCount); - spec.put("maxReplicaCount", maxReplicas); - if (autoscaling.minReplicas() == 0) { - spec.put("idleReplicaCount", 0); - } - spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); - spec.put("pollingInterval", 30); - - // Advanced scaling policy: scale down one pod at a time for graceful drain - spec.put("advanced", Map.of( - "horizontalPodAutoscalerConfig", Map.of( - "behavior", Map.of( - "scaleDown", Map.of( - "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), - "policies", List.of(Map.of( - "type", "Pods", - "value", 1, - "periodSeconds", 60 - )) - ), - "scaleUp", Map.of( - "stabilizationWindowSeconds", 60, - "policies", List.of(Map.of( - "type", "Percent", - "value", 100, - "periodSeconds", 60 - )) - ) - ) - ) - )); - - // Triggers: Prometheus for hs2_active_sessions + CPU fallback (only when CPU requests defined) - // "or vector(0)" ensures the query returns 0 (not empty) when HS2 has no pods. - List> triggers = new ArrayList<>(); - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_active_sessions", - "query", String.format( - "avg(hs2_active_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - )); - if (hiveCluster.getSpec().hiveServer2().resources() != null) { - // activationValue prevents idle JVM CPU from keeping the ScaledObject active. - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", "75", - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); - } - spec.put("triggers", triggers); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("keda.sh/v1alpha1") - .withKind("ScaledObject") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hiveserver2-scaledobject"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java deleted file mode 100644 index 13b218986e67..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java +++ /dev/null @@ -1,87 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.HiveServer2Spec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the Kubernetes Service for HiveServer2 (Thrift and WebUI ports). */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=hiveserver2," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class HiveServer2ServiceDependent - extends HiveDependentResource { - - public HiveServer2ServiceDependent() { - super(Service.class); - } - - @Override - protected Service desired(HiveCluster hiveCluster, - Context context) { - HiveServer2Spec hs2 = hiveCluster.getSpec().hiveServer2(); - int thriftPort = ConfigUtils.getInt(hs2.configOverrides(), - ConfigUtils.HIVE_SERVER2_THRIFT_PORT_KEY, - null, ConfigUtils.HIVE_SERVER2_THRIFT_PORT_DEFAULT); - int httpPort = ConfigUtils.getInt(hs2.configOverrides(), - ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, - null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); - int webUiPort = ConfigUtils.getInt(hs2.configOverrides(), - ConfigUtils.HIVE_SERVER2_WEBUI_PORT_KEY, - null, ConfigUtils.HIVE_SERVER2_WEBUI_PORT_DEFAULT); - - return new ServiceBuilder() - .withNewMetadata() - .withName(hiveCluster.getMetadata().getName() + "-hiveserver2") - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, - HiveServer2DeploymentDependent.COMPONENT)) - .endMetadata() - .withNewSpec() - .withType(hs2.serviceType()) - .withSelector(Labels.selectorForComponent(hiveCluster, - HiveServer2DeploymentDependent.COMPONENT)) - .addNewPort() - .withName("thrift") - .withPort(thriftPort) - .withTargetPort(new IntOrString(thriftPort)) - .endPort() - .addNewPort() - .withName("http") - .withPort(httpPort) - .withTargetPort(new IntOrString(httpPort)) - .endPort() - .addNewPort() - .withName("webui") - .withPort(webUiPort) - .withTargetPort(new IntOrString(webUiPort)) - .endPort() - .endSpec() - .build(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java new file mode 100644 index 000000000000..f3a064f48b95 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -0,0 +1,165 @@ +/* + * 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.hive.kubernetes.operator.dependent; + +import io.fabric8.kubernetes.api.model.IntOrString; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; +import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.config.informer.Informer; +import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Unified Kubernetes Service dependent for all Hive components. + * Subclassed per component to define component-specific service type and ports. + */ +public abstract class HiveServiceDependent + extends HiveDependentResource { + + private final String component; + + protected HiveServiceDependent(String component) { + super(Service.class); + this.component = component; + } + + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return primary.getMetadata().getName() + "-" + component; + } + + @Override + protected Service desired(HiveCluster hiveCluster, + Context context) { + ServiceBuilder builder = new ServiceBuilder() + .withNewMetadata() + .withName(hiveCluster.getMetadata().getName() + "-" + component) + .withNamespace(hiveCluster.getMetadata().getNamespace()) + .withLabels(Labels.forComponent(hiveCluster, component)) + .endMetadata() + .withNewSpec() + .withSelector(Labels.selectorForComponent(hiveCluster, component)) + .endSpec(); + customizeSpec(builder, hiveCluster); + return builder.build(); + } + + /** Subclasses override to set service type and add ports. */ + protected abstract void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster); + + /** HiveServer2 Service: configurable type, thrift + http + webui ports. */ + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=hiveserver2," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class HiveServer2 extends HiveServiceDependent { + public HiveServer2() { super("hiveserver2"); } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + var hs2 = hiveCluster.getSpec().hiveServer2(); + int thriftPort = ConfigUtils.getInt(hs2.configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_PORT_DEFAULT); + int httpPort = ConfigUtils.getInt(hs2.configOverrides(), + ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); + int webUiPort = ConfigUtils.getInt(hs2.configOverrides(), + ConfigUtils.HIVE_SERVER2_WEBUI_PORT_KEY, + null, ConfigUtils.HIVE_SERVER2_WEBUI_PORT_DEFAULT); + builder.editSpec() + .withType(hs2.serviceType()) + .addNewPort().withName("thrift").withProtocol("TCP") + .withPort(thriftPort).withTargetPort(new IntOrString(thriftPort)).endPort() + .addNewPort().withName("http").withProtocol("TCP") + .withPort(httpPort).withTargetPort(new IntOrString(httpPort)).endPort() + .addNewPort().withName("webui").withProtocol("TCP") + .withPort(webUiPort).withTargetPort(new IntOrString(webUiPort)).endPort() + .endSpec(); + } + } + + /** Metastore Service: ClusterIP, thrift + rest ports. */ + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=metastore," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class Metastore extends HiveServiceDependent { + public Metastore() { super("metastore"); } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + int thriftPort = ConfigUtils.getInt( + hiveCluster.getSpec().metastore().configOverrides(), + ConfigUtils.METASTORE_THRIFT_PORT_KEY, + ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, + ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); + builder.editSpec() + .withType("ClusterIP") + .addNewPort().withName("thrift").withProtocol("TCP") + .withPort(thriftPort).withTargetPort(new IntOrString(thriftPort)).endPort() + .addNewPort().withName("rest").withProtocol("TCP") + .withPort(9001).withTargetPort(new IntOrString(9001)).endPort() + .endSpec(); + } + } + + /** LLAP headless Service: required by StatefulSet for stable DNS. */ + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=llap," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class Llap extends HiveServiceDependent { + public Llap() { super("llap"); } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + builder.editSpec() + .withClusterIP("None") + .addNewPort().withName("management").withProtocol("TCP") + .withPort(15004).withTargetPort(new IntOrString(15004)).endPort() + .addNewPort().withName("shuffle").withProtocol("TCP") + .withPort(15551).withTargetPort(new IntOrString(15551)).endPort() + .addNewPort().withName("web").withProtocol("TCP") + .withPort(15002).withTargetPort(new IntOrString(15002)).endPort() + .endSpec(); + } + } + + /** TezAM headless Service: required by StatefulSet for stable DNS. */ + @KubernetesDependent( + informer = @Informer(labelSelector = "app.kubernetes.io/component=tezam," + + "app.kubernetes.io/managed-by=hive-kubernetes-operator") + ) + public static class TezAm extends HiveServiceDependent { + public TezAm() { super("tezam"); } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + builder.editSpec() + .withClusterIP("None") + .endSpec(); + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapConfigMapDependent.java deleted file mode 100644 index 2ad6955dadb8..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapConfigMapDependent.java +++ /dev/null @@ -1,68 +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.hive.kubernetes.operator.dependent; - -import java.util.Map; - -import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.api.model.ConfigMapBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; -import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the llap-daemon-site.xml ConfigMap for LLAP daemons. */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=llap," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class LlapConfigMapDependent - extends HiveDependentResource { - - public static final String COMPONENT = "llap"; - - public LlapConfigMapDependent() { - super(ConfigMap.class); - } - - @Override - protected ConfigMap desired(HiveCluster hiveCluster, - Context context) { - Map props = - HiveConfigBuilder.getLlapDaemonSite(hiveCluster.getSpec()); - - return new ConfigMapBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .endMetadata() - .addToData("llap-daemon-site.xml", - HadoopXmlBuilder.buildXml(props)) - .build(); - } - - /** Returns the ConfigMap resource name for this HiveCluster. */ - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-llap-config"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java deleted file mode 100644 index 1f077751aa61..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapPdbDependent.java +++ /dev/null @@ -1,62 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * PodDisruptionBudget for LLAP daemons. - * Ensures at least one LLAP daemon remains available during voluntary disruptions - * to prevent query failures and cache loss. - */ -public class LlapPdbDependent - extends CRUDKubernetesDependentResource { - - public LlapPdbDependent() { - super(PodDisruptionBudget.class); - } - - @Override - protected PodDisruptionBudget desired(HiveCluster hiveCluster, - Context context) { - return new PodDisruptionBudgetBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "llap")) - .endMetadata() - .withNewSpec() - .withMinAvailable(new IntOrString(1)) - .withNewSelector() - .withMatchLabels(Labels.selectorForComponent(hiveCluster, "llap")) - .endSelector() - .endSpec() - .build(); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-llap-pdb"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java deleted file mode 100644 index 7f6886a594df..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapScaledObjectDependent.java +++ /dev/null @@ -1,158 +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.hive.kubernetes.operator.dependent; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA ScaledObject for LLAP daemon autoscaling. - *

- * Scale-up trigger: - * - NumQueuedRequests > 0 for 1 minute (queue non-empty means all executors are busy) - *

- * Scale-down trigger: - * - NumExecutorsAvailable == NumExecutors (daemon completely idle) - *

- * Cooldown: configurable (default 900s / 15 minutes — scaling down destroys in-memory cache) - */ -public class LlapScaledObjectDependent extends HiveGenericDependentResource { - - public LlapScaledObjectDependent() { - super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().llap().autoscaling(); - int maxReplicas = hiveCluster.getSpec().llap().replicas(); - String targetName = hiveCluster.getMetadata().getName() + "-llap"; - - Map spec = new HashMap<>(); - spec.put("scaleTargetRef", Map.of( - "apiVersion", "apps/v1", - "kind", "StatefulSet", - "name", targetName - )); - // KEDA requires idleReplicaCount < minReplicaCount. - // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). - int minReplicaCount = Math.max(1, autoscaling.minReplicas()); - spec.put("minReplicaCount", minReplicaCount); - spec.put("maxReplicaCount", maxReplicas); - if (autoscaling.minReplicas() == 0) { - spec.put("idleReplicaCount", 0); - } - spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); - spec.put("pollingInterval", 5); - - // LLAP scale-up is aggressive: when queries need daemons, scale immediately to max. - // Scale down is slow (1 pod per cooldown) to preserve in-memory cache. - spec.put("advanced", Map.of( - "horizontalPodAutoscalerConfig", Map.of( - "behavior", Map.of( - "scaleDown", Map.of( - "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), - "policies", List.of(Map.of( - "type", "Pods", - "value", 1, - "periodSeconds", autoscaling.cooldownSeconds() - )) - ), - "scaleUp", Map.of( - "stabilizationWindowSeconds", 0, - "policies", List.of(Map.of( - "type", "Pods", - "value", maxReplicas, - "periodSeconds", 15 - )) - ) - ) - ) - )); - - // Triggers: - // 1. Prometheus for NumQueuedRequests — drives proportional scaling. - // More queued requests = more LLAP daemons needed. Scales up to max. - // 2. HS2 open sessions — activation only (wake from 0→1). - // Threshold set to maxReplicas so desired = 1/max ≈ 1 (never drives above min). - // activationThreshold=0 ensures any session activates the ScaledObject. - // - // Scale-down: HPA policy removes 1 pod per cooldown period (preserves cache). - // Idle (all sessions closed + no queued requests): after cooldownPeriod → 0. - // "or vector(0)" ensures queries return 0 (not empty) when pods don't exist. - String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; - String namespace = hiveCluster.getMetadata().getNamespace(); - spec.put("triggers", List.of( - Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "llap_num_queued_requests", - "query", String.format( - "avg(hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - namespace, targetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - ), - Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_open_sessions_activation", - "query", String.format( - "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(maxReplicas), - "activationThreshold", "0" - ) - ) - )); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("keda.sh/v1alpha1") - .withKind("ScaledObject") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "llap")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-llap-scaledobject"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapServiceDependent.java deleted file mode 100644 index 108f29347a97..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapServiceDependent.java +++ /dev/null @@ -1,77 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages the headless Kubernetes Service for LLAP daemons. - * Required by the StatefulSet for stable DNS entries and ZooKeeper registration. - */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=llap," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class LlapServiceDependent - extends HiveDependentResource { - - public LlapServiceDependent() { - super(Service.class); - } - - @Override - protected Service desired(HiveCluster hiveCluster, - Context context) { - return new ServiceBuilder() - .withNewMetadata() - .withName(hiveCluster.getMetadata().getName() + "-llap") - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, - LlapStatefulSetDependent.COMPONENT)) - .endMetadata() - .withNewSpec() - .withClusterIP("None") - .withSelector(Labels.selectorForComponent(hiveCluster, - LlapStatefulSetDependent.COMPONENT)) - .addNewPort() - .withName("management") - .withPort(15004) - .withTargetPort(new IntOrString(15004)) - .endPort() - .addNewPort() - .withName("shuffle") - .withPort(15551) - .withTargetPort(new IntOrString(15551)) - .endPort() - .addNewPort() - .withName("web") - .withPort(15002) - .withTargetPort(new IntOrString(15002)) - .endPort() - .endSpec() - .build(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 35fcbb8ac171..7bd1473afefc 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -26,8 +26,6 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.Lifecycle; -import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder; @@ -38,7 +36,6 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -60,6 +57,12 @@ public LlapStatefulSetDependent() { super(StatefulSet.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -87,13 +90,13 @@ protected StatefulSet desired(HiveCluster hiveCluster, List ports = new ArrayList<>(); ports.add(new ContainerPortBuilder() - .withName("management").withContainerPort(15004).build()); + .withName("management").withContainerPort(15004).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("shuffle").withContainerPort(15551).build()); + .withName("shuffle").withContainerPort(15551).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("web").withContainerPort(15002).build()); + .withName("web").withContainerPort(15002).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("output").withContainerPort(15003).build()); + .withName("output").withContainerPort(15003).withProtocol("TCP").build()); Probe readinessProbe = buildTcpProbe(15004, llap.readinessProbe(), 15, 10, 3); @@ -109,8 +112,8 @@ protected StatefulSet desired(HiveCluster hiveCluster, List volumes = new ArrayList<>(); volumes.add(buildProjectedConfigVolume("llap-config", - LlapConfigMapDependent.resourceName(hiveCluster), - HadoopConfigMapDependent.resourceName(hiveCluster))); + HiveConfigMapDependent.Llap.resourceName(hiveCluster), + HiveConfigMapDependent.Hadoop.resourceName(hiveCluster))); List initContainers = new ArrayList<>(); addExternalJars(spec.image(), spec.externalJars(), @@ -130,18 +133,12 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getLlapDaemonSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled and the StatefulSet already exists, preserve the current - // replica count (managed by KEDA/HPA). On initial creation: - // - minReplicas == 0: start at 0, KEDA scales up when hs2_active_sessions > 0 - // - minReplicas > 0: start at configured replicas - boolean autoscalingEnabled = llap.autoscaling() != null && llap.autoscaling().isEnabled(); - Integer replicas = llap.replicas(); - if (autoscalingEnabled) { - int initialReplicas = llap.autoscaling().minReplicas() == 0 ? 0 : llap.replicas(); - replicas = getSecondaryResource(hiveCluster, context) - .map(s -> s.getSpec().getReplicas()) - .orElse(initialReplicas); - } + // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). + AutoscalingSpec llapAutoscaling = llap.autoscaling(); + int initialReplicas = llapAutoscaling != null && llapAutoscaling.minReplicas() == 0 + ? 0 : llap.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, llapAutoscaling, llap.replicas(), initialReplicas); StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() @@ -183,70 +180,24 @@ protected StatefulSet desired(HiveCluster hiveCluster, statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); // Graceful scale-down: poll JMX Exporter (port 9404) until all executors idle. - // Uses flat Prometheus text format — same metrics KEDA reads — not brittle JSON parsing. if (autoscaling.isEnabled()) { - String preStopScript = String.join("\n", - "#!/bin/bash", - "echo '[preStop] Waiting for LLAP executors to become idle (polling localhost:9404/metrics)...'", - "RETRIES=0", - "while true; do", - " RESPONSE=$(curl -sf http://localhost:9404/metrics)", - " if [ $? -ne 0 ]; then", - " RETRIES=$((RETRIES+1))", - " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", - " if [ $RETRIES -ge 6 ]; then", - " echo '[preStop] JMX Exporter not responding after 60s. Proceeding with shutdown.'", - " break", - " fi", - " sleep 10; continue", - " fi", - " AVAILABLE=$(echo \"$RESPONSE\" | grep '^hadoop_llapdaemon_executornumexecutorsavailable{' | awk '{print $2}')", - " TOTAL=$(echo \"$RESPONSE\" | grep '^hadoop_llapdaemon_executornumexecutors{' | awk '{print $2}')", - " if [ -z \"$AVAILABLE\" ] || [ -z \"$TOTAL\" ]; then", - " echo '[preStop] WARNING: LLAP executor metrics not found. JMX Exporter may not be configured.'", - " break", - " fi", - " if [ \"${AVAILABLE%.*}\" -ge \"${TOTAL%.*}\" ] 2>/dev/null; then", - " echo '[preStop] All executors idle. Shutting down.'", - " break", - " fi", - " echo \"[preStop] Executors available=$AVAILABLE / total=$TOTAL — waiting...\"", - " RETRIES=0", - " sleep 10", - "done"); - Lifecycle lifecycle = new LifecycleBuilder() - .withNewPreStop() - .withNewExec() - .withCommand("/bin/bash", "-c", preStopScript) - .endExec() - .endPreStop() - .build(); - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); - statefulSet.getSpec().getTemplate().getSpec() - .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); - // Prometheus scrape annotations for JMX Exporter metrics endpoint - statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/scrape", "true"); - statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); - statefulSet.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/path", "/metrics"); + String preStopScript = buildDualMetricDrainScript( + "Waiting for LLAP executors to become idle", + "hadoop_llapdaemon_executornumexecutorsavailable{", "AVAILABLE", + "hadoop_llapdaemon_executornumexecutors{", "TOTAL", + "LLAP executor metrics not found. JMX Exporter may not be configured.", + "All executors idle. Shutting down.", + "Executors available=$AVAILABLE / total=$TOTAL \u2014 waiting...", + 10, 6); + applyAutoscalingLifecycle( + statefulSet.getSpec().getTemplate().getSpec(), + statefulSet.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds()); } - if (spec.volumes() != null) { - statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); - } - if (spec.volumeMounts() != null) { - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(spec.volumeMounts()); - } - if (llap.extraVolumes() != null) { - statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(llap.extraVolumes()); - } - if (llap.extraVolumeMounts() != null) { - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(llap.extraVolumeMounts()); - } + appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), + spec.volumes(), spec.volumeMounts(), + llap.extraVolumes(), llap.extraVolumeMounts()); return statefulSet; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreConfigMapDependent.java deleted file mode 100644 index b429335f76e0..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreConfigMapDependent.java +++ /dev/null @@ -1,67 +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.hive.kubernetes.operator.dependent; - -import java.util.Map; - -import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.api.model.ConfigMapBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; -import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the metastore-site.xml ConfigMap for the Hive Metastore. */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=metastore," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class MetastoreConfigMapDependent - extends HiveDependentResource { - - public static final String COMPONENT = "metastore"; - - public MetastoreConfigMapDependent() { - super(ConfigMap.class); - } - - @Override - protected ConfigMap desired(HiveCluster hiveCluster, - Context context) { - Map props = - HiveConfigBuilder.getMetastoreSite(hiveCluster.getSpec()); - - return new ConfigMapBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .endMetadata() - .addToData("metastore-site.xml", HadoopXmlBuilder.buildXml(props)) - .build(); - } - - /** Returns the ConfigMap resource name for this HiveCluster. */ - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-metastore-config"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index e1f88caacb63..ce16cc17eeaf 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -26,8 +26,6 @@ import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.Lifecycle; -import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.Probe; import io.fabric8.kubernetes.api.model.Volume; import io.fabric8.kubernetes.api.model.VolumeMount; @@ -59,6 +57,12 @@ public MetastoreDeploymentDependent() { super(Deployment.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -82,9 +86,9 @@ protected Deployment desired(HiveCluster hiveCluster, ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); List ports = new ArrayList<>(); ports.add(new ContainerPortBuilder() - .withName("thrift").withContainerPort(thriftPort).build()); + .withName("thrift").withContainerPort(thriftPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("rest").withContainerPort(9001).build()); + .withName("rest").withContainerPort(9001).withProtocol("TCP").build()); Probe readinessProbe = buildTcpProbe(thriftPort, spec.metastore().readinessProbe(), 15, 10, 3); Probe livenessProbe = buildTcpProbe(thriftPort, spec.metastore().livenessProbe(), 60, 30, 5); @@ -124,18 +128,12 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getMetastoreSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled and the Deployment already exists, preserve the current - // replica count (managed by KEDA/HPA). On initial creation, start at minReplicas - // and let KEDA scale up based on load. - boolean autoscalingEnabled = spec.metastore().autoscaling() != null - && spec.metastore().autoscaling().isEnabled(); - Integer replicas = spec.metastore().replicas(); - if (autoscalingEnabled) { - int initialReplicas = Math.max(1, spec.metastore().autoscaling().minReplicas()); - replicas = getSecondaryResource(hiveCluster, context) - .map(d -> d.getSpec().getReplicas()) - .orElse(initialReplicas); - } + // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). + AutoscalingSpec msAutoscaling = spec.metastore().autoscaling(); + int initialReplicas = msAutoscaling != null + ? Math.max(1, msAutoscaling.minReplicas()) : spec.metastore().replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, msAutoscaling, spec.metastore().replicas(), initialReplicas); Deployment deployment = new DeploymentBuilder() .withNewMetadata() @@ -178,70 +176,21 @@ protected Deployment desired(HiveCluster hiveCluster, deployment.getSpec().getTemplate().getSpec(), selectorLabels); // Graceful scale-down: poll JMX Exporter (port 9404) for open_connections to drain. - // K8s removes the pod from Service Endpoints on termination, so no new requests arrive. - // Uses flat Prometheus text format — same metric KEDA reads — not brittle JSON parsing. if (autoscaling.isEnabled()) { - String preStopScript = String.join("\n", - "#!/bin/bash", - "echo '[preStop] Waiting for open connections to drain (polling localhost:9404/metrics)...'", - "RETRIES=0", - "while true; do", - " RESPONSE=$(curl -sf http://localhost:9404/metrics)", - " if [ $? -ne 0 ]; then", - " RETRIES=$((RETRIES+1))", - " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", - " if [ $RETRIES -ge 6 ]; then", - " echo '[preStop] JMX Exporter not responding after 30s. Proceeding with shutdown.'", - " break", - " fi", - " sleep 5; continue", - " fi", - " CONNS=$(echo \"$RESPONSE\" | grep '^hive_metastore_open_connections ' | awk '{print $2}')", - " if [ -z \"$CONNS\" ]; then", - " echo '[preStop] WARNING: hive_metastore_open_connections metric not found. JMX Exporter may not be configured.'", - " break", - " fi", - " if [ \"${CONNS%.*}\" -le 0 ] 2>/dev/null; then", - " echo '[preStop] All connections drained. Shutting down.'", - " break", - " fi", - " echo \"[preStop] hive_metastore_open_connections=$CONNS — waiting...\"", - " RETRIES=0", - " sleep 5", - "done"); - Lifecycle lifecycle = new LifecycleBuilder() - .withNewPreStop() - .withNewExec() - .withCommand("/bin/bash", "-c", preStopScript) - .endExec() - .endPreStop() - .build(); - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); - deployment.getSpec().getTemplate().getSpec() - .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); - // Prometheus scrape annotations for JMX Exporter metrics endpoint - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/scrape", "true"); - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); - deployment.getSpec().getTemplate().getMetadata().getAnnotations() - .put("prometheus.io/path", "/metrics"); + String preStopScript = buildDrainScript( + "Waiting for open connections to drain", + "hive_metastore_open_connections", "CONNS", + "All connections drained. Shutting down.", + 5, 6, null); + applyAutoscalingLifecycle( + deployment.getSpec().getTemplate().getSpec(), + deployment.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds()); } - if (spec.volumes() != null) { - deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); - } - if (spec.volumeMounts() != null) { - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(spec.volumeMounts()); - } - if (spec.metastore().extraVolumes() != null) { - deployment.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.metastore().extraVolumes()); - } - if (spec.metastore().extraVolumeMounts() != null) { - deployment.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(spec.metastore().extraVolumeMounts()); - } + appendUserVolumes(deployment.getSpec().getTemplate().getSpec(), + spec.volumes(), spec.volumeMounts(), + spec.metastore().extraVolumes(), spec.metastore().extraVolumeMounts()); return deployment; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java deleted file mode 100644 index e177e1e60138..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastorePdbDependent.java +++ /dev/null @@ -1,62 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * PodDisruptionBudget for Hive Metastore. - * Ensures at least one Metastore pod remains available during voluntary disruptions - * to prevent catalog access failures. - */ -public class MetastorePdbDependent - extends CRUDKubernetesDependentResource { - - public MetastorePdbDependent() { - super(PodDisruptionBudget.class); - } - - @Override - protected PodDisruptionBudget desired(HiveCluster hiveCluster, - Context context) { - return new PodDisruptionBudgetBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "metastore")) - .endMetadata() - .withNewSpec() - .withMinAvailable(new IntOrString(1)) - .withNewSelector() - .withMatchLabels(Labels.selectorForComponent(hiveCluster, "metastore")) - .endSelector() - .endSpec() - .build(); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-metastore-pdb"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java deleted file mode 100644 index 58263318468f..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreScaledObjectDependent.java +++ /dev/null @@ -1,152 +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.hive.kubernetes.operator.dependent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA ScaledObject for Hive Metastore autoscaling. - *

- * Scale-up triggers (OR): - * - Open connections exceed threshold (Prometheus) - * - Pod CPU > 75% - *

- * Scale-down triggers (AND): - * - CPU < activationValue - * - Open connections at 0 - *

- * Cooldown: configurable (default 300s / 5 minutes) - * Guardrail: replicas should be set based on backend DB max_connections. - */ -public class MetastoreScaledObjectDependent extends HiveGenericDependentResource { - - public MetastoreScaledObjectDependent() { - super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().metastore().autoscaling(); - int maxReplicas = hiveCluster.getSpec().metastore().replicas(); - String targetName = hiveCluster.getMetadata().getName() + "-metastore"; - - // Threshold = max threads per pod (from metastore-site config or default 1000). - // KEDA divides total open_connections by threshold to determine desired replicas. - int maxThreads = ConfigUtils.getInt( - hiveCluster.getSpec().metastore().configOverrides(), - ConfigUtils.METASTORE_SERVER_MAX_THREADS_KEY, - ConfigUtils.METASTORE_SERVER_MAX_THREADS_HIVE_KEY, - ConfigUtils.METASTORE_SERVER_MAX_THREADS_DEFAULT); - - Map spec = new HashMap<>(); - spec.put("scaleTargetRef", Map.of( - "apiVersion", "apps/v1", - "kind", "Deployment", - "name", targetName - )); - spec.put("minReplicaCount", autoscaling.minReplicas()); - spec.put("maxReplicaCount", maxReplicas); - spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); - spec.put("pollingInterval", 30); - - spec.put("advanced", Map.of( - "horizontalPodAutoscalerConfig", Map.of( - "behavior", Map.of( - "scaleDown", Map.of( - "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), - "policies", List.of(Map.of( - "type", "Pods", - "value", 1, - "periodSeconds", 60 - )) - ), - "scaleUp", Map.of( - "stabilizationWindowSeconds", 120, - "policies", List.of(Map.of( - "type", "Percent", - "value", 50, - "periodSeconds", 60 - )) - ) - ) - ) - )); - - // Triggers: Prometheus for open connections + CPU (only when CPU requests are defined) - // "or vector(0)" ensures the query returns 0 (not empty) when no pods match. - List> triggers = new ArrayList<>(); - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hive_metastore_open_connections", - "query", String.format( - "sum(hive_metastore_open_connections{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName), - "threshold", String.valueOf(maxThreads), - "activationThreshold", "0" - ) - )); - if (hiveCluster.getSpec().metastore().resources() != null) { - // activationValue prevents idle JVM CPU from keeping the ScaledObject active. - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", "75", - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); - } - spec.put("triggers", triggers); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("keda.sh/v1alpha1") - .withKind("ScaledObject") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "metastore")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-metastore-scaledobject"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreServiceDependent.java deleted file mode 100644 index 2620a24e01d7..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreServiceDependent.java +++ /dev/null @@ -1,75 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** Manages the Kubernetes Service for the Hive Metastore (Thrift + REST ports). */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=metastore," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class MetastoreServiceDependent - extends HiveDependentResource { - - public MetastoreServiceDependent() { - super(Service.class); - } - - @Override - protected Service desired(HiveCluster hiveCluster, - Context context) { - int thriftPort = ConfigUtils.getInt( - hiveCluster.getSpec().metastore().configOverrides(), - ConfigUtils.METASTORE_THRIFT_PORT_KEY, - ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, - ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); - return new ServiceBuilder() - .withNewMetadata() - .withName(hiveCluster.getMetadata().getName() + "-metastore") - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, - MetastoreDeploymentDependent.COMPONENT)) - .endMetadata() - .withNewSpec() - .withType("ClusterIP") - .withSelector(Labels.selectorForComponent(hiveCluster, - MetastoreDeploymentDependent.COMPONENT)) - .addNewPort() - .withName("thrift") - .withPort(thriftPort) - .withTargetPort(new IntOrString(thriftPort)) - .endPort() - .addNewPort() - .withName("rest") - .withPort(9001) - .withTargetPort(new IntOrString(9001)) - .endPort() - .endSpec() - .build(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java index a23c0c477436..25d0eb39a0f9 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java @@ -53,6 +53,12 @@ public SchemaInitJobDependent() { super(Job.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected Job desired(HiveCluster hiveCluster, Context context) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/ScratchPvcDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/ScratchPvcDependent.java index 6a645f043574..230ba47edd13 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/ScratchPvcDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/ScratchPvcDependent.java @@ -55,6 +55,12 @@ public ScratchPvcDependent() { super(PersistentVolumeClaim.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected PersistentVolumeClaim desired(HiveCluster hiveCluster, Context context) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java deleted file mode 100644 index 13fc6343cad0..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmPdbDependent.java +++ /dev/null @@ -1,61 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.IntOrString; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudget; -import io.fabric8.kubernetes.api.model.policy.v1.PodDisruptionBudgetBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * PodDisruptionBudget for Tez Application Masters. - * Ensures at least one Tez AM remains available in the warm pool during voluntary disruptions. - */ -public class TezAmPdbDependent - extends CRUDKubernetesDependentResource { - - public TezAmPdbDependent() { - super(PodDisruptionBudget.class); - } - - @Override - protected PodDisruptionBudget desired(HiveCluster hiveCluster, - Context context) { - return new PodDisruptionBudgetBuilder() - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "tezam")) - .endMetadata() - .withNewSpec() - .withMinAvailable(new IntOrString(1)) - .withNewSelector() - .withMatchLabels(Labels.selectorForComponent(hiveCluster, "tezam")) - .endSelector() - .endSpec() - .build(); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-tezam-pdb"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java deleted file mode 100644 index 731eb6f08d97..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmScaledObjectDependent.java +++ /dev/null @@ -1,200 +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.hive.kubernetes.operator.dependent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA ScaledObject for Tez Application Master autoscaling. - *

- * Tez AMs run in a warm pool (StatefulSet). An unclaimed AM sits idle; - * a claimed AM actively orchestrates a query DAG and consumes CPU. - *

- * Scale-up trigger: - * - Pod CPU > 60% across the StatefulSet (most AMs claimed and working) - *

- * Scale-down trigger: - * - Pod CPU < 10% (many idle unclaimed AMs) - *

- * Cooldown: configurable (default 600s / 10 minutes) - */ -public class TezAmScaledObjectDependent extends HiveGenericDependentResource { - - public TezAmScaledObjectDependent() { - super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().tezAm().autoscaling(); - int maxReplicas = hiveCluster.getSpec().tezAm().replicas(); - String targetName = hiveCluster.getMetadata().getName() + "-tezam"; - - Map spec = new HashMap<>(); - spec.put("scaleTargetRef", Map.of( - "apiVersion", "apps/v1", - "kind", "StatefulSet", - "name", targetName - )); - // KEDA requires idleReplicaCount < minReplicaCount. - // For scale-to-zero: min=1 (minimum when active), idle=0 (scale to zero when idle). - // For non-zero min: just set minReplicaCount (no idle needed). - int minReplicaCount = Math.max(1, autoscaling.minReplicas()); - spec.put("minReplicaCount", minReplicaCount); - spec.put("maxReplicaCount", maxReplicas); - if (autoscaling.minReplicas() == 0) { - spec.put("idleReplicaCount", 0); - } - spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); - spec.put("pollingInterval", 5); - - spec.put("advanced", Map.of( - "horizontalPodAutoscalerConfig", Map.of( - "behavior", Map.of( - "scaleDown", Map.of( - "stabilizationWindowSeconds", autoscaling.cooldownSeconds(), - "policies", List.of(Map.of( - "type", "Pods", - "value", 1, - "periodSeconds", 60 - )) - ), - "scaleUp", Map.of( - "stabilizationWindowSeconds", 60, - "policies", List.of(Map.of( - "type", "Pods", - "value", 2, - "periodSeconds", 30 - )) - ) - ) - ) - )); - - // Triggers: - // 1. CPU utilization — the primary proportional scaler for warm-pool Tez AMs - // (only included when container has CPU requests defined, required by KEDA) - // 2. HS2 cross-component activation: when HS2 has open sessions, - // TezAM should be available (enables wake-from-zero) - // - // When CPU IS available: CPU drives proportional scaling, HS2 trigger is activation-only - // (threshold set to maxReplicas so it never dominates the HPA calculation). - // When CPU is NOT available: tez_session_pending_tasks drives proportional scaling - // (real query demand — tasks waiting for AM slots), with HS2 sessions for activation only. - String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; - String namespace = hiveCluster.getMetadata().getNamespace(); - List> triggers = new ArrayList<>(); - if (hiveCluster.getSpec().tezAm().resources() != null) { - // CPU drives proportional scaling; activationValue prevents idle JVM CPU - // from keeping the ScaledObject permanently "active" (blocks scale-to-zero). - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", String.valueOf(autoscaling.scaleUpThreshold()), - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); - // Activation-only: (sessions > bool 0) returns 0 or 1, with threshold=maxReplicas - // ensures desired = ceil(1/max) = 1 — never drives replica count above min. - // activationThreshold=0 ensures any open session wakes TezAM from zero. - // Uses hs2_open_sessions (connection-level) not hs2_active_sessions (query-level). - // "or vector(0)" ensures the query returns 0 (not empty) when HS2 has no pods. - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_open_sessions_activation", - "query", String.format( - "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(maxReplicas), - "activationThreshold", "0" - ) - )); - } else { - // No CPU available: use tez_session_pending_tasks for proportional scaling. - // This metric reflects real query demand (tasks waiting for AM slots), unlike - // hs2_open_sessions which includes zombie/idle sessions from ungracefully closed clients. - // Threshold: scaleUpThreshold interpreted as pending-tasks-per-AM (default 60 when - // using CPU mode, but for pending tasks a lower value like 5-10 is recommended). - // "or vector(0)" ensures the query returns 0 when HS2 has no pods. - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "tez_session_pending_tasks", - "query", String.format( - "sum(tez_session_pending_tasks{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - )); - // Activation-only: (sessions > bool 0) returns 0 or 1, with threshold=maxReplicas - // ensures desired = ceil(1/max) = 1 — never drives replica count above min. - // activationThreshold=0 ensures any open session wakes TezAM from zero. - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_open_sessions_activation", - "query", String.format( - "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(maxReplicas), - "activationThreshold", "0" - ) - )); - } - spec.put("triggers", triggers); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("keda.sh/v1alpha1") - .withKind("ScaledObject") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, "tezam")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-tezam-scaledobject"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmServiceDependent.java deleted file mode 100644 index 781685286038..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmServiceDependent.java +++ /dev/null @@ -1,62 +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.hive.kubernetes.operator.dependent; - -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.config.informer.Informer; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages the headless Kubernetes Service for Tez Application Master. - * Required by the StatefulSet for stable DNS entries so that - * HiveServer2 can resolve TezAM pod hostnames for RPC communication. - */ -@KubernetesDependent( - informer = @Informer(labelSelector = "app.kubernetes.io/component=tezam," - + "app.kubernetes.io/managed-by=hive-kubernetes-operator") -) -public class TezAmServiceDependent - extends HiveDependentResource { - - public TezAmServiceDependent() { - super(Service.class); - } - - @Override - protected Service desired(HiveCluster hiveCluster, - Context context) { - return new ServiceBuilder() - .withNewMetadata() - .withName(hiveCluster.getMetadata().getName() + "-tezam") - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, - TezAmStatefulSetDependent.COMPONENT)) - .endMetadata() - .withNewSpec() - .withClusterIP("None") - .withSelector(Labels.selectorForComponent(hiveCluster, - TezAmStatefulSetDependent.COMPONENT)) - .endSpec() - .build(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index 56e60ca10403..01a0509f99ca 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -24,10 +24,7 @@ import io.fabric8.kubernetes.api.model.Container; import io.fabric8.kubernetes.api.model.ContainerPort; -import io.fabric8.kubernetes.api.model.ContainerPortBuilder; import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.Lifecycle; -import io.fabric8.kubernetes.api.model.LifecycleBuilder; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder; import io.javaoperatorsdk.operator.api.reconciler.Context; @@ -62,6 +59,12 @@ public TezAmStatefulSetDependent() { super(StatefulSet.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -103,8 +106,8 @@ protected StatefulSet desired(HiveCluster hiveCluster, List volumes = new ArrayList<>(); volumes.add(buildProjectedConfigVolume("hive-config", - HiveServer2ConfigMapDependent.resourceName(hiveCluster), - HadoopConfigMapDependent.resourceName(hiveCluster))); + HiveConfigMapDependent.HiveServer2.resourceName(hiveCluster), + HiveConfigMapDependent.Hadoop.resourceName(hiveCluster))); volumes.add(new io.fabric8.kubernetes.api.model.VolumeBuilder() .withName("scratch") .withNewPersistentVolumeClaim() @@ -133,18 +136,12 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled and the StatefulSet already exists, preserve the current - // replica count (managed by KEDA/HPA). On initial creation: - // - minReplicas == 0: start at 0, KEDA scales up when hs2_active_sessions > 0 - // - minReplicas > 0: start at configured replicas - boolean autoscalingEnabled = tezAm.autoscaling() != null && tezAm.autoscaling().isEnabled(); - Integer replicas = tezAm.replicas(); - if (autoscalingEnabled) { - int initialReplicas = tezAm.autoscaling().minReplicas() == 0 ? 0 : tezAm.replicas(); - replicas = getSecondaryResource(hiveCluster, context) - .map(s -> s.getSpec().getReplicas()) - .orElse(initialReplicas); - } + // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). + AutoscalingSpec tezAmAutoscaling = tezAm.autoscaling(); + int initialReplicas = tezAmAutoscaling != null && tezAmAutoscaling.minReplicas() == 0 + ? 0 : tezAm.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, tezAmAutoscaling, tezAm.replicas(), initialReplicas); StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() @@ -185,63 +182,21 @@ protected StatefulSet desired(HiveCluster hiveCluster, statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); // Graceful scale-down: poll JMX Exporter (port 9404) for DAGsRunning to reach 0. - // K8s removes the pod from Service Endpoints, so HS2 won't assign new DAGs to this AM. - // We read from the same Prometheus-format endpoint that KEDA uses — flat text, not brittle JSON. if (autoscaling.isEnabled()) { - String preStopScript = String.join("\n", - "#!/bin/bash", - "echo '[preStop] Waiting for active DAGs to complete (polling localhost:9404/metrics)...'", - "RETRIES=0", - "while true; do", - " RESPONSE=$(curl -sf http://localhost:9404/metrics)", - " if [ $? -ne 0 ]; then", - " RETRIES=$((RETRIES+1))", - " echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\"", - " if [ $RETRIES -ge 6 ]; then", - " echo '[preStop] JMX Exporter not responding after 60s. Proceeding with shutdown.'", - " break", - " fi", - " sleep 10; continue", - " fi", - " DAGS=$(echo \"$RESPONSE\" | grep '^tez_am_dagsrunning ' | awk '{print $2}')", - " if [ -z \"$DAGS\" ]; then", - " echo '[preStop] WARNING: tez_am_dagsrunning metric not found. JMX Exporter may not be configured.'", - " break", - " fi", - " if [ \"${DAGS%.*}\" -le 0 ] 2>/dev/null; then", - " echo '[preStop] No active DAGs. Safe to terminate Tez AM.'", - " break", - " fi", - " echo \"[preStop] tez_am_dagsrunning=$DAGS — waiting...\"", - " RETRIES=0", - " sleep 10", - "done"); - Lifecycle lifecycle = new LifecycleBuilder() - .withNewPreStop() - .withNewExec() - .withCommand("/bin/bash", "-c", preStopScript) - .endExec() - .endPreStop() - .build(); - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).setLifecycle(lifecycle); - statefulSet.getSpec().getTemplate().getSpec() - .setTerminationGracePeriodSeconds((long) autoscaling.gracePeriodSeconds()); + String preStopScript = buildDrainScript( + "Waiting for active DAGs to complete", + "tez_am_dagsrunning", "DAGS", + "No active DAGs. Safe to terminate Tez AM.", + 10, 6, null); + applyAutoscalingLifecycle( + statefulSet.getSpec().getTemplate().getSpec(), + statefulSet.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds()); } - if (spec.volumes() != null) { - statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(spec.volumes()); - } - if (spec.volumeMounts() != null) { - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(spec.volumeMounts()); - } - if (tezAm.extraVolumes() != null) { - statefulSet.getSpec().getTemplate().getSpec().getVolumes().addAll(tezAm.extraVolumes()); - } - if (tezAm.extraVolumeMounts() != null) { - statefulSet.getSpec().getTemplate().getSpec().getContainers().get(0).getVolumeMounts() - .addAll(tezAm.extraVolumeMounts()); - } + appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), + spec.volumes(), spec.volumeMounts(), + tezAm.extraVolumes(), tezAm.extraVolumeMounts()); return statefulSet; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java deleted file mode 100644 index bf14dac91e54..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2AutoscalingCondition.java +++ /dev/null @@ -1,41 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for HiveServer2 autoscaling dependent resources. - * Returns true only when spec.hiveServer2.autoscaling.enabled is true. - */ -public class HiveServer2AutoscalingCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().hiveServer2().autoscaling().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java deleted file mode 100644 index 9c01942e4a1c..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2MetricScalingCondition.java +++ /dev/null @@ -1,44 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; - -/** - * Activation condition for HiveServer2 Prometheus-based ScaledObject. - * Returns true when autoscaling is enabled AND minReplicas > 0. - * When minReplicas == 0, the HTTPScaledObject is used instead (scale-to-zero). - */ -public class HiveServer2MetricScalingCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - AutoscalingSpec autoscaling = primary.getSpec().hiveServer2().autoscaling(); - return autoscaling.isEnabled() && autoscaling.minReplicas() > 0; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java deleted file mode 100644 index 81f07269e9c9..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java +++ /dev/null @@ -1,60 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.apps.Deployment; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Precondition for HiveServer2 Deployment. - * If Metastore is external, proceed immediately. - * If managed, wait for Metastore pods to be ready. - */ -public class HiveServer2Precondition implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - - if (!primary.getSpec().metastore().isEnabled()) { - return true; - } - - // When autoscaling is enabled, wait for minReplicas (KEDA manages scaling beyond that). - // Without autoscaling, wait for all configured replicas. - int desiredReplicas; - if (primary.getSpec().metastore().autoscaling().isEnabled()) { - desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); - } else { - desiredReplicas = primary.getSpec().metastore().replicas(); - } - return context.getSecondaryResources(Deployment.class).stream() - .filter(d -> d.getMetadata().getName().equals(primary.getMetadata().getName() + "-metastore")) - .findFirst() - .map(deployment -> deployment.getStatus() != null - && deployment.getStatus().getReadyReplicas() != null - && deployment.getStatus().getReadyReplicas() >= desiredReplicas) - .orElse(false); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java deleted file mode 100644 index 7ae91b8f7b8f..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2ScaleToZeroCondition.java +++ /dev/null @@ -1,44 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; - -/** - * Activation condition for HiveServer2 scale-to-zero (HTTPScaledObject). - * Returns true when autoscaling is enabled AND minReplicas == 0. - * Requires the KEDA HTTP Add-on to be installed in the cluster. - */ -public class HiveServer2ScaleToZeroCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - AutoscalingSpec autoscaling = primary.getSpec().hiveServer2().autoscaling(); - return autoscaling.isEnabled() && autoscaling.minReplicas() == 0; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java deleted file mode 100644 index f4e097786b08..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapAutoscalingCondition.java +++ /dev/null @@ -1,42 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for LLAP autoscaling dependent resources. - * Returns true only when spec.llap.enabled is true and spec.llap.autoscaling.enabled is true. - */ -public class LlapAutoscalingCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().llap().isEnabled() - && primary.getSpec().llap().autoscaling().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapEnabledCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapEnabledCondition.java deleted file mode 100644 index a113c50efbff..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/LlapEnabledCondition.java +++ /dev/null @@ -1,41 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for LLAP dependent resources. - * Returns true only when spec.llap.enabled is true. - */ -public class LlapEnabledCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().llap().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java deleted file mode 100644 index a0ac83d8a423..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreAutoscalingCondition.java +++ /dev/null @@ -1,42 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for Metastore autoscaling dependent resources. - * Returns true only when spec.metastore.autoscaling.enabled is true and metastore is managed. - */ -public class MetastoreAutoscalingCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().metastore().isEnabled() - && primary.getSpec().metastore().autoscaling().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreEnabledCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreEnabledCondition.java deleted file mode 100644 index b1cb4139ac96..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreEnabledCondition.java +++ /dev/null @@ -1,39 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for Metastore dependent resources. - * Returns true only when spec.metastore.enabled is true. - */ -public class MetastoreEnabledCondition implements Condition { - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().metastore().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java deleted file mode 100644 index 58885c6e8865..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java +++ /dev/null @@ -1,56 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.apps.Deployment; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Ready condition that checks whether the Metastore Deployment has the - * desired number of ready replicas. Used to gate HiveServer2 Deployment. - */ -public class MetastoreReadyCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - if (!primary.getSpec().metastore().isEnabled()) { - return true; - } - // When autoscaling is enabled, wait for minReplicas (KEDA manages scaling beyond that). - // Without autoscaling, wait for all configured replicas. - int desiredReplicas; - if (primary.getSpec().metastore().autoscaling().isEnabled()) { - desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); - } else { - desiredReplicas = primary.getSpec().metastore().replicas(); - } - return dependentResource.getSecondaryResource(primary, context) - .map(deployment -> deployment.getStatus() != null - && deployment.getStatus().getReadyReplicas() != null - && deployment.getStatus().getReadyReplicas() >= desiredReplicas) - .orElse(false); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/SchemaJobCompletedCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/SchemaJobCompletedCondition.java deleted file mode 100644 index 1b0b44318596..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/SchemaJobCompletedCondition.java +++ /dev/null @@ -1,48 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.batch.v1.Job; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Ready condition that checks whether the schema initialization Job - * has completed successfully. Used to gate Metastore Deployment creation. - */ -public class SchemaJobCompletedCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - if (!primary.getSpec().metastore().isEnabled()) { - return true; - } - return dependentResource.getSecondaryResource(primary, context) - .map(job -> job.getStatus() != null - && job.getStatus().getSucceeded() != null - && job.getStatus().getSucceeded() >= 1) - .orElse(false); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java deleted file mode 100644 index a2ed23cbbadc..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmAutoscalingCondition.java +++ /dev/null @@ -1,42 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for Tez AM autoscaling dependent resources. - * Returns true only when spec.tezAm.enabled is true and spec.tezAm.autoscaling.enabled is true. - */ -public class TezAmAutoscalingCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().tezAm().isEnabled() - && primary.getSpec().tezAm().autoscaling().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmEnabledCondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmEnabledCondition.java deleted file mode 100644 index 85ae7e45dbdb..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/TezAmEnabledCondition.java +++ /dev/null @@ -1,41 +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.hive.kubernetes.operator.dependent.condition; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.dependent.DependentResource; -import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Activation condition for Tez AM dependent resources. - * Returns true only when spec.tezAm.enabled is true. - */ -public class TezAmEnabledCondition - implements Condition { - - @Override - public boolean isMet( - DependentResource dependentResource, - HiveCluster primary, - Context context) { - return primary.getSpec().tezAm().isEnabled(); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java index 40dd8a771203..1897582bd18e 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java @@ -78,6 +78,14 @@ public record HiveClusterSpec( public HiveClusterSpec { Objects.requireNonNull(zookeeper, "zookeeper must be provided in the HiveCluster spec"); + metastore = metastore != null ? metastore : new MetastoreSpec( + 1, null, null, null, null, null, null, true, null, null, null, null); + hiveServer2 = hiveServer2 != null ? hiveServer2 : new HiveServer2Spec( + 1, null, null, null, null, null, null, null, null, null); + llap = llap != null ? llap : new LlapSpec( + 1, null, null, null, null, true, null, null, null, null, null); + tezAm = tezAm != null ? tezAm : new TezAmSpec( + 1, null, null, null, null, true, null, null, null); envVars = envVars != null ? envVars : List.of(); externalJars = externalJars != null ? externalJars : List.of(); volumes = volumes != null ? volumes : List.of(); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index c973145b0080..6a0008b9147d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 60, 10, 600, 120); + false, 0, 5, 10, 600, 120); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index e621f7065a54..71453ef0335b 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -35,43 +35,6 @@ import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; import io.javaoperatorsdk.operator.api.reconciler.Reconciler; import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; -import io.javaoperatorsdk.operator.api.reconciler.Workflow; -import io.javaoperatorsdk.operator.api.reconciler.dependent.Dependent; -import org.apache.hive.kubernetes.operator.dependent.HadoopConfigMapDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2ConfigMapDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2DeploymentDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2PdbDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2HttpScaledObjectDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2ScaledObjectDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2ServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapConfigMapDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapPdbDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapScaledObjectDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapStatefulSetDependent; -import org.apache.hive.kubernetes.operator.dependent.MetastoreConfigMapDependent; -import org.apache.hive.kubernetes.operator.dependent.MetastoreDeploymentDependent; -import org.apache.hive.kubernetes.operator.dependent.MetastorePdbDependent; -import org.apache.hive.kubernetes.operator.dependent.MetastoreScaledObjectDependent; -import org.apache.hive.kubernetes.operator.dependent.MetastoreServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.SchemaInitJobDependent; -import org.apache.hive.kubernetes.operator.dependent.ScratchPvcDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmPdbDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmScaledObjectDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmStatefulSetDependent; -import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2AutoscalingCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2MetricScalingCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2Precondition; -import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2ScaleToZeroCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.LlapAutoscalingCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.LlapEnabledCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreAutoscalingCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreEnabledCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.MetastoreReadyCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.SchemaJobCompletedCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.TezAmAutoscalingCondition; -import org.apache.hive.kubernetes.operator.dependent.condition.TezAmEnabledCondition; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterStatus; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; @@ -83,74 +46,21 @@ * Orchestrates all dependent resources with proper dependency ordering. */ @ControllerConfiguration -@Workflow(dependents = { - // --- ConfigMap dependents --- - @Dependent(name = "hadoop-configmap", type = HadoopConfigMapDependent.class), - @Dependent(name = "metastore-configmap", type = MetastoreConfigMapDependent.class, - activationCondition = MetastoreEnabledCondition.class), - @Dependent(name = "hiveserver2-configmap", type = HiveServer2ConfigMapDependent.class), - // --- Job dependents --- - @Dependent(name = "schema-init-job", type = SchemaInitJobDependent.class, dependsOn = {"metastore-configmap", - "hadoop-configmap"}, readyPostcondition = SchemaJobCompletedCondition.class, - activationCondition = MetastoreEnabledCondition.class), - // --- Deployment dependents --- - @Dependent(name = "metastore-deployment", type = MetastoreDeploymentDependent.class, dependsOn = { - "schema-init-job"}, readyPostcondition = MetastoreReadyCondition.class, - activationCondition = MetastoreEnabledCondition.class), - // --- Service dependents --- - @Dependent(name = "metastore-service", type = MetastoreServiceDependent.class, dependsOn = { - "metastore-configmap"}, activationCondition = MetastoreEnabledCondition.class), - @Dependent(name = "hiveserver2-deployment", type = HiveServer2DeploymentDependent.class, dependsOn = { - "hiveserver2-configmap", "hadoop-configmap"}, reconcilePrecondition = HiveServer2Precondition.class), - @Dependent(name = "hiveserver2-service", type = HiveServer2ServiceDependent.class, dependsOn = { - "hiveserver2-configmap"}), - // --- LLAP (conditional) --- - @Dependent(name = "llap-configmap", type = LlapConfigMapDependent.class, - activationCondition = LlapEnabledCondition.class), - @Dependent(name = "llap-statefulset", type = LlapStatefulSetDependent.class, dependsOn = {"llap-configmap", - "hadoop-configmap"}, activationCondition = LlapEnabledCondition.class), - @Dependent(name = "llap-service", type = LlapServiceDependent.class, - activationCondition = LlapEnabledCondition.class), - // --- TezAM (conditional) --- - @Dependent(name = "scratch-pvc", type = ScratchPvcDependent.class, - activationCondition = TezAmEnabledCondition.class), - @Dependent(name = "tezam-service", type = TezAmServiceDependent.class, - activationCondition = TezAmEnabledCondition.class), - @Dependent(name = "tezam-statefulset", type = TezAmStatefulSetDependent.class, dependsOn = {"hiveserver2-configmap", - "hadoop-configmap", "tezam-service", "scratch-pvc"}, activationCondition = TezAmEnabledCondition.class), - // --- Autoscaling: KEDA ScaledObjects (conditional) --- - @Dependent(name = "hs2-scaledobject", type = HiveServer2ScaledObjectDependent.class, dependsOn = { - "hiveserver2-deployment"}, activationCondition = HiveServer2MetricScalingCondition.class), - @Dependent(name = "hs2-httpso", type = HiveServer2HttpScaledObjectDependent.class, dependsOn = { - "hiveserver2-deployment"}, activationCondition = HiveServer2ScaleToZeroCondition.class), - @Dependent(name = "metastore-scaledobject", type = MetastoreScaledObjectDependent.class, dependsOn = { - "metastore-deployment"}, activationCondition = MetastoreAutoscalingCondition.class), - @Dependent(name = "llap-scaledobject", type = LlapScaledObjectDependent.class, dependsOn = { - "llap-statefulset"}, activationCondition = LlapAutoscalingCondition.class), - @Dependent(name = "tezam-scaledobject", type = TezAmScaledObjectDependent.class, dependsOn = { - "tezam-statefulset"}, activationCondition = TezAmAutoscalingCondition.class), - // --- Autoscaling: PodDisruptionBudgets (conditional) --- - @Dependent(name = "hs2-pdb", type = HiveServer2PdbDependent.class, dependsOn = { - "hiveserver2-deployment"}, activationCondition = HiveServer2AutoscalingCondition.class), - @Dependent(name = "metastore-pdb", type = MetastorePdbDependent.class, dependsOn = { - "metastore-deployment"}, activationCondition = MetastoreAutoscalingCondition.class), - @Dependent(name = "llap-pdb", type = LlapPdbDependent.class, dependsOn = { - "llap-statefulset"}, activationCondition = LlapAutoscalingCondition.class), - @Dependent(name = "tezam-pdb", type = TezAmPdbDependent.class, dependsOn = { - "tezam-statefulset"}, activationCondition = TezAmAutoscalingCondition.class)}) public class HiveClusterReconciler implements Reconciler { private static final Logger LOG = LoggerFactory.getLogger(HiveClusterReconciler.class); @Override public UpdateControl reconcile(HiveCluster resource, Context context) { - LOG.debug("Reconciling HiveCluster: {}/{}", resource.getMetadata().getNamespace(), - resource.getMetadata().getName()); + LOG.debug("Reconciling HiveCluster: {}/{} generation={}", + resource.getMetadata().getNamespace(), + resource.getMetadata().getName(), + resource.getMetadata().getGeneration()); HiveClusterStatus existingStatus = resource.getStatus(); HiveClusterStatus newStatus = buildStatus(resource, context, existingStatus); - if (Objects.equals(existingStatus, newStatus)) { + if (statusEqualsIgnoringTimestamps(existingStatus, newStatus)) { return UpdateControl.noUpdate(); } @@ -161,8 +71,8 @@ public UpdateControl reconcile(HiveCluster resource, Context updateErrorStatus(HiveCluster resource, Context context, Exception e) { - LOG.error("Error reconciling HiveCluster: {}/{}", resource.getMetadata().getNamespace(), - resource.getMetadata().getName(), e); + LOG.error("Error reconciling HiveCluster: {}/{} - {}", resource.getMetadata().getNamespace(), + resource.getMetadata().getName(), e.getMessage(), e); HiveClusterStatus status = resource.getStatus() != null ? resource.getStatus() : new HiveClusterStatus(); @@ -312,14 +222,86 @@ private Condition buildCondition(String type, String conditionStatus, condition.setReason(reason); condition.setMessage(message); - // Preserve lastTransitionTime when the condition status has not changed + // Preserve lastTransitionTime from ANY existing condition of this type + // (regardless of status) to avoid generating new timestamps on every + // reconcile which would cause an infinite status-patch loop. String preservedTime = existingConditions.stream() - .filter(c -> type.equals(c.getType()) && conditionStatus.equals(c.getStatus())) + .filter(c -> type.equals(c.getType())) .map(Condition::getLastTransitionTime) .findFirst() .orElse(null); - condition.setLastTransitionTime(preservedTime != null ? preservedTime : Instant.now().toString()); + if (preservedTime != null) { + // Only update the timestamp if the status actually changed + String oldStatus = existingConditions.stream() + .filter(c -> type.equals(c.getType())) + .map(Condition::getStatus) + .findFirst() + .orElse(null); + if (conditionStatus.equals(oldStatus)) { + condition.setLastTransitionTime(preservedTime); + } else { + condition.setLastTransitionTime(Instant.now().toString()); + } + } else { + condition.setLastTransitionTime(Instant.now().toString()); + } return condition; } + + /** + * Compares two HiveClusterStatus objects ignoring condition timestamps. + * This prevents infinite reconciliation loops caused by informer cache lag: + * after a status patch, the informer may still have the old status, causing + * the next reconcile to see a "different" status (new timestamp vs old) and + * patch again, perpetuating the loop. + */ + private boolean statusEqualsIgnoringTimestamps(HiveClusterStatus a, HiveClusterStatus b) { + if (a == b) { + return true; + } + if (a == null || b == null) { + return false; + } + if (!Objects.equals(a.getObservedGeneration(), b.getObservedGeneration())) { + return false; + } + if (!Objects.equals(a.getMetastore(), b.getMetastore())) { + return false; + } + if (!Objects.equals(a.getHiveServer2(), b.getHiveServer2())) { + return false; + } + if (!Objects.equals(a.getLlap(), b.getLlap())) { + return false; + } + if (!Objects.equals(a.getTezAm(), b.getTezAm())) { + return false; + } + // Compare conditions by type+status+reason+message, ignoring lastTransitionTime + return conditionsEqualIgnoringTime(a.getConditions(), b.getConditions()); + } + + private boolean conditionsEqualIgnoringTime(List a, List b) { + if (a == b) { + return true; + } + if (a == null || b == null) { + return a == null && b == null; + } + if (a.size() != b.size()) { + return false; + } + for (int i = 0; i < a.size(); i++) { + Condition ca = a.get(i); + Condition cb = b.get(i); + if (!Objects.equals(ca.getType(), cb.getType()) + || !Objects.equals(ca.getStatus(), cb.getStatus()) + || !Objects.equals(ca.getReason(), cb.getReason()) + || !Objects.equals(ca.getMessage(), cb.getMessage())) { + return false; + } + } + return true; + } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java new file mode 100644 index 000000000000..46aa53890573 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java @@ -0,0 +1,290 @@ +/* + * 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.hive.kubernetes.operator.reconciler; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import io.fabric8.kubernetes.api.model.apps.Deployment; +import io.javaoperatorsdk.operator.api.config.dependent.DependentResourceSpec; +import io.javaoperatorsdk.operator.api.config.workflow.WorkflowSpec; +import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; +import org.apache.hive.kubernetes.operator.dependent.HiveConfigMapDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServer2DeploymentDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServer2InterceptorRouteDependent; +import org.apache.hive.kubernetes.operator.dependent.HivePdbDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveScaledObjectDependent; +import org.apache.hive.kubernetes.operator.dependent.HiveServiceDependent; +import org.apache.hive.kubernetes.operator.dependent.LlapStatefulSetDependent; +import org.apache.hive.kubernetes.operator.dependent.MetastoreDeploymentDependent; +import org.apache.hive.kubernetes.operator.dependent.SchemaInitJobDependent; +import org.apache.hive.kubernetes.operator.dependent.ScratchPvcDependent; +import org.apache.hive.kubernetes.operator.dependent.TezAmStatefulSetDependent; +import org.apache.hive.kubernetes.operator.model.HiveCluster; + +/** + * Programmatic workflow specification for the Hive Kubernetes Operator. + * Replaces the annotation-based {@code @Workflow} on the reconciler with + * explicit {@link DependentResourceSpec} entries and inline lambda conditions. + * This eliminates 12 single-method condition wrapper classes. + */ +public final class HiveWorkflowSpec implements WorkflowSpec { + + private static final Condition METASTORE_ENABLED = + (dr, primary, ctx) -> primary.getSpec().metastore().isEnabled(); + + private static final Condition LLAP_ENABLED = + (dr, primary, ctx) -> primary.getSpec().llap().isEnabled(); + + private static final Condition TEZAM_ENABLED = + (dr, primary, ctx) -> primary.getSpec().tezAm().isEnabled(); + + private static final Condition METASTORE_AUTOSCALING = + (dr, primary, ctx) -> primary.getSpec().metastore().isEnabled() + && primary.getSpec().metastore().autoscaling().isEnabled(); + + private static final Condition LLAP_AUTOSCALING = + (dr, primary, ctx) -> primary.getSpec().llap().isEnabled() + && primary.getSpec().llap().autoscaling().isEnabled(); + + private static final Condition TEZAM_AUTOSCALING = + (dr, primary, ctx) -> primary.getSpec().tezAm().isEnabled() + && primary.getSpec().tezAm().autoscaling().isEnabled(); + + private static final Condition HS2_AUTOSCALING = + (dr, primary, ctx) -> primary.getSpec().hiveServer2().autoscaling().isEnabled(); + + private static final Condition HS2_SCALE_TO_ZERO = + (dr, primary, ctx) -> primary.getSpec().hiveServer2().autoscaling().isEnabled() + && primary.getSpec().hiveServer2().autoscaling().minReplicas() == 0; + + // SPECS must be declared AFTER all conditions to avoid static init order issues. + private static final List SPECS = buildSpecs(); + + @SuppressWarnings({"rawtypes", "unchecked"}) + private static List buildSpecs() { + List specs = new ArrayList<>(); + + // --- ConfigMap dependents --- + specs.add(new DependentResourceSpec( + HiveConfigMapDependent.Hadoop.class, "hadoop-configmap", + Set.of(), null, null, null, null, null)); + + specs.add(new DependentResourceSpec( + HiveConfigMapDependent.Metastore.class, "metastore-configmap", + Set.of(), null, null, null, METASTORE_ENABLED, null)); + + specs.add(new DependentResourceSpec( + HiveConfigMapDependent.HiveServer2.class, "hiveserver2-configmap", + Set.of(), null, null, null, null, null)); + + // --- Job dependents --- + specs.add(new DependentResourceSpec( + SchemaInitJobDependent.class, "schema-init-job", + Set.of("metastore-configmap", "hadoop-configmap"), + schemaJobCompleted(), null, null, METASTORE_ENABLED, null)); + + // --- Deployment dependents --- + specs.add(new DependentResourceSpec( + MetastoreDeploymentDependent.class, "metastore-deployment", + Set.of("schema-init-job"), + metastoreReady(), null, null, METASTORE_ENABLED, null)); + + // --- Service dependents --- + specs.add(new DependentResourceSpec( + HiveServiceDependent.Metastore.class, "metastore-service", + Set.of("metastore-configmap"), + null, null, null, METASTORE_ENABLED, null)); + + specs.add(new DependentResourceSpec( + HiveServer2DeploymentDependent.class, "hiveserver2-deployment", + Set.of("hiveserver2-configmap", "hadoop-configmap"), + null, hs2Precondition(), null, null, null)); + + specs.add(new DependentResourceSpec( + HiveServiceDependent.HiveServer2.class, "hiveserver2-service", + Set.of("hiveserver2-configmap"), + null, null, null, null, null)); + + // --- LLAP (conditional) --- + specs.add(new DependentResourceSpec( + HiveConfigMapDependent.Llap.class, "llap-configmap", + Set.of(), null, null, null, LLAP_ENABLED, null)); + + specs.add(new DependentResourceSpec( + LlapStatefulSetDependent.class, "llap-statefulset", + Set.of("llap-configmap", "hadoop-configmap"), + null, null, null, LLAP_ENABLED, null)); + + specs.add(new DependentResourceSpec( + HiveServiceDependent.Llap.class, "llap-service", + Set.of(), null, null, null, LLAP_ENABLED, null)); + + // --- TezAM (conditional) --- + specs.add(new DependentResourceSpec( + ScratchPvcDependent.class, "scratch-pvc", + Set.of(), null, null, null, TEZAM_ENABLED, null)); + + specs.add(new DependentResourceSpec( + HiveServiceDependent.TezAm.class, "tezam-service", + Set.of(), null, null, null, TEZAM_ENABLED, null)); + + specs.add(new DependentResourceSpec( + TezAmStatefulSetDependent.class, "tezam-statefulset", + Set.of("hiveserver2-configmap", "hadoop-configmap", "tezam-service", "scratch-pvc"), + null, null, null, TEZAM_ENABLED, null)); + + specs.add(new DependentResourceSpec( + HiveScaledObjectDependent.HiveServer2.class, "hs2-scaledobject", + Set.of("hiveserver2-deployment"), + null, HS2_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HiveServer2InterceptorRouteDependent.class, "hs2-interceptor-route", + Set.of("hiveserver2-deployment"), + null, HS2_SCALE_TO_ZERO, null, null, null)); + + specs.add(new DependentResourceSpec( + HiveScaledObjectDependent.Metastore.class, "metastore-scaledobject", + Set.of("metastore-deployment"), + null, METASTORE_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HiveScaledObjectDependent.Llap.class, "llap-scaledobject", + Set.of("llap-statefulset", "hs2-scaledobject"), + null, LLAP_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HiveScaledObjectDependent.TezAm.class, "tezam-scaledobject", + Set.of("tezam-statefulset", "hs2-scaledobject"), + null, TEZAM_AUTOSCALING, null, null, null)); + + // --- Autoscaling: PodDisruptionBudgets (conditional) --- + specs.add(new DependentResourceSpec( + HivePdbDependent.HiveServer2.class, "hs2-pdb", + Set.of("hiveserver2-deployment"), + null, HS2_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HivePdbDependent.Metastore.class, "metastore-pdb", + Set.of("metastore-deployment"), + null, METASTORE_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HivePdbDependent.Llap.class, "llap-pdb", + Set.of("llap-statefulset"), + null, LLAP_AUTOSCALING, null, null, null)); + + specs.add(new DependentResourceSpec( + HivePdbDependent.TezAm.class, "tezam-pdb", + Set.of("tezam-statefulset"), + null, TEZAM_AUTOSCALING, null, null, null)); + + return Collections.unmodifiableList(specs); + } + + /** + * Ready postcondition: schema initialization Job must complete successfully + * before the Metastore Deployment is created. + */ + private static Condition schemaJobCompleted() { + return (dependentResource, primary, context) -> { + if (!primary.getSpec().metastore().isEnabled()) { + return true; + } + return dependentResource.getSecondaryResource(primary, context) + .map(job -> { + var j = (io.fabric8.kubernetes.api.model.batch.v1.Job) job; + return j.getStatus() != null + && j.getStatus().getSucceeded() != null + && j.getStatus().getSucceeded() >= 1; + }) + .orElse(false); + }; + } + + /** + * Ready postcondition: Metastore Deployment must have the desired number + * of ready replicas before downstream dependents proceed. + */ + private static Condition metastoreReady() { + return (dependentResource, primary, context) -> { + if (!primary.getSpec().metastore().isEnabled()) { + return true; + } + int desiredReplicas; + if (primary.getSpec().metastore().autoscaling().isEnabled()) { + desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); + } else { + desiredReplicas = primary.getSpec().metastore().replicas(); + } + return dependentResource.getSecondaryResource(primary, context) + .map(resource -> { + var deployment = (Deployment) resource; + return deployment.getStatus() != null + && deployment.getStatus().getReadyReplicas() != null + && deployment.getStatus().getReadyReplicas() >= desiredReplicas; + }) + .orElse(false); + }; + } + + /** + * Reconcile precondition for HiveServer2: if Metastore is managed, + * wait for it to be ready before reconciling HS2. + */ + private static Condition hs2Precondition() { + return (dependentResource, primary, context) -> { + if (!primary.getSpec().metastore().isEnabled()) { + return true; + } + int desiredReplicas; + if (primary.getSpec().metastore().autoscaling().isEnabled()) { + desiredReplicas = Math.max(1, primary.getSpec().metastore().autoscaling().minReplicas()); + } else { + desiredReplicas = primary.getSpec().metastore().replicas(); + } + return context.getSecondaryResources(Deployment.class).stream() + .filter(d -> d.getMetadata().getName().equals( + primary.getMetadata().getName() + "-metastore")) + .findFirst() + .map(deployment -> deployment.getStatus() != null + && deployment.getStatus().getReadyReplicas() != null + && deployment.getStatus().getReadyReplicas() >= desiredReplicas) + .orElse(false); + }; + } + + @Override + public List getDependentResourceSpecs() { + return SPECS; + } + + @Override + public boolean isExplicitInvocation() { + return false; + } + + @Override + public boolean handleExceptionsInReconciler() { + return true; + } +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestVectorMapJoinOuterGenerateResultOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestVectorMapJoinOuterGenerateResultOperator.java index 35553d9cb445..85e6882d4d68 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestVectorMapJoinOuterGenerateResultOperator.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestVectorMapJoinOuterGenerateResultOperator.java @@ -54,7 +54,6 @@ */ class TestVectorMapJoinOuterGenerateResultOperator { - /** Concrete subclass that exposes the generateOuterNulls* methods to tests. */ private static final class TestableOuterOp extends VectorMapJoinOuterGenerateResultOperator { @Override protected String getLoggingPrefix() { From 2c094568998bbe88b693f5d1c3d92b3385bc9e9f Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 28 May 2026 19:36:45 +0530 Subject: [PATCH 03/22] Fix CPU Utilization Scaling --- packaging/src/kubernetes/README.md | 79 +++++++++------- .../crds/hiveclusters.hive.apache.org-v1.yml | 72 ++++++++++----- .../hive-operator/templates/hivecluster.yaml | 18 ++++ .../kubernetes/helm/hive-operator/values.yaml | 9 +- .../dependent/HiveScaledObjectDependent.java | 89 ++++++++++++------- .../dependent/HiveServiceDependent.java | 17 ++-- .../operator/model/spec/AutoscalingSpec.java | 13 ++- .../operator/model/spec/HiveServer2Spec.java | 2 +- .../operator/model/spec/LlapSpec.java | 2 +- .../operator/model/spec/MetastoreSpec.java | 2 +- .../operator/model/spec/TezAmSpec.java | 2 +- 11 files changed, 203 insertions(+), 102 deletions(-) diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index ac2a4d0e9584..afe55dbe9139 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -647,7 +647,7 @@ without creating a conflicting second ScaledObject. The HS2 ScaledObject combines three trigger types in a single resource: - **Prometheus trigger** (`hs2_open_sessions`) — session-aware scaling -- **CPU trigger** — load-based scaling when resources are configured +- **CPU trigger** (`AverageValue` in millicores) — load-based scaling when `targetCpuValue` is configured - **external-push trigger** — wake-from-zero via the KEDA HTTP Add-on interceptor The `InterceptorRoute` CRD (`http.keda.sh/v1beta1`) configures only the interceptor @@ -791,69 +791,82 @@ When autoscaling is enabled, the operator automatically: | **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available (scaling trigger) | | **Tez AM** | Standard K8s CPU metrics or `tez_session_pending_tasks` (from HS2) | CPU utilization or pending task count (scaling trigger) | -### CPU-Based Scaling and Resource Requests +### CPU-Based Scaling -The operator includes a **CPU utilization trigger** in the ScaledObject for HS2, Metastore, -and Tez AM. KEDA's CPU trigger uses the `Utilization` metric type, which is defined as a -percentage of the container's CPU request. This means **the container must have a CPU request -defined** for the trigger to work. +The operator can include a **CPU trigger** in the ScaledObject for HS2, Metastore, and Tez AM. +The trigger uses KEDA's `AverageValue` metric type with **absolute millicore targets** that +you specify directly. This handles burstable QoS pods correctly — unlike `Utilization` +(which measures against the CPU request), `AverageValue` uses actual CPU consumption in +absolute terms, so pods with a small request but high limit won't show perpetual >100% +utilization that prevents scale-down. + +**The CPU trigger is opt-in:** it is only added to the ScaledObject when you explicitly set +both `targetCpuValue` and `activationCpuValue` in the autoscaling config. If omitted, the +operator relies solely on the Prometheus-based trigger (sessions, connections, etc.). **How it works:** -- The CPU trigger scales up when pod CPU utilization exceeds `scaleUpThreshold`% of the CPU request -- The `scaleDownThreshold` configures the **activation threshold** — below this CPU%, the - trigger is completely inactive (doesn't participate in scaling decisions) -- Both the CPU trigger and the Prometheus-based trigger (sessions/connections) are evaluated - independently — if **either** exceeds its threshold, the component scales up (OR logic) -- Scale-down only happens when **both** triggers agree load is low (all below threshold) +- `targetCpuValue` — the average CPU per pod (e.g., `"1500m"` or `"1"`) that triggers scale-up +- `activationCpuValue` — below this CPU value, the trigger is completely inactive + (doesn't participate in scaling decisions at all) +- Both the CPU trigger and the Prometheus-based trigger are evaluated independently — + if **either** exceeds its threshold, the component scales up (OR logic) +- Scale-down only happens when **both** triggers agree load is low +- The component must also have `resources` defined on its pods; if `targetCpuValue` is set + but `resources` is missing, the operator logs a warning and skips the CPU trigger -This means a long-running CPU-intensive query will keep the pod scaled even if there's -only one session open. Conversely, many idle sessions will keep it scaled even at low CPU. +**Example:** With `targetCpuValue: "1600m"` and `activationCpuValue: "400m"`, KEDA scales up +when average pod CPU exceeds 1600m and considers the trigger inactive below 400m. -If you enable autoscaling without setting `resources` for that component, the operator -will omit the CPU trigger and rely solely on the Prometheus-based trigger. For Tez AM -specifically, without CPU resources the operator uses `tez_session_pending_tasks` (queued -tasks waiting for AM slots) as the proportional scaler — this reflects real query demand -rather than connection count, avoiding spurious scale-ups from idle or zombie sessions. +For Tez AM specifically, without CPU targets the operator uses `tez_session_pending_tasks` +(queued tasks waiting for AM slots) as the proportional scaler — this reflects real query +demand rather than connection count, avoiding spurious scale-ups from idle sessions. -To get both Prometheus and CPU-based scaling, set `resources` on the component: +To enable both Prometheus and CPU-based scaling: ```yaml cluster: hiveServer2: resources: - requestsCpu: "1" # Required for CPU-based autoscaling + requestsCpu: "500m" + limitsCpu: "2" requestsMemory: "2Gi" autoscaling: enabled: true - scaleDownThreshold: 30 # CPU trigger inactive below 30% (default) + scaleUpThreshold: 1 # scale up when avg sessions > 1 per pod + targetCpuValue: "1600m" # scale up when avg CPU > 1600m per pod + activationCpuValue: "400m" # CPU trigger inactive below 400m metastore: resources: - requestsCpu: "500m" # Required for CPU-based autoscaling + requestsCpu: "500m" + limitsCpu: "1" requestsMemory: "1Gi" autoscaling: enabled: true + targetCpuValue: "750m" + activationCpuValue: "200m" tezAm: resources: - requestsCpu: "500m" # Required for CPU-based autoscaling + requestsCpu: "250m" + limitsCpu: "1" requestsMemory: "1Gi" autoscaling: enabled: true - scaleUpThreshold: 60 # For TezAM with resources, this IS the CPU target % - scaleDownThreshold: 10 # CPU trigger inactive below 10% + targetCpuValue: "600m" + activationCpuValue: "100m" ``` | Setting | Effect on CPU trigger | |---------|----------------------| -| `resources.requestsCpu` | **Enables** the CPU trigger (required) | -| `scaleUpThreshold` | CPU target % — scales up when utilization exceeds this (default 80) | -| `scaleDownThreshold` | Activation value — CPU trigger ignored below this % (default 30) | +| `targetCpuValue` | Absolute CPU target (e.g., `"1500m"` or `"1"`). **Required** to enable CPU trigger. | +| `activationCpuValue` | CPU below which trigger is inactive. **Required** with targetCpuValue. | +| `resources` | Pod resources must be defined — operator warns and skips CPU trigger otherwise. | > **Note:** LLAP scaling uses only Prometheus triggers (total busy slots) -> and does not include a CPU trigger, so LLAP does not require `resources` to -> be set for autoscaling to work. +> and does not include a CPU trigger, so LLAP does not require `targetCpuValue` +> for autoscaling to work. ### Helm Values Reference (Autoscaling) @@ -1073,7 +1086,9 @@ kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hi | `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling for this component | | `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (HS2 requires KEDA HTTP Add-on) | | `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (sessions for HS2, connections for HMS, busy slots for LLAP, pending tasks or CPU% for TezAM) | -| `cluster..autoscaling.scaleDownThreshold` | `30` | CPU activation threshold below which the CPU trigger is inactive | +| `cluster..autoscaling.scaleDownThreshold` | `30` | Prometheus metric threshold for scale-down (component-specific) | +| `cluster..autoscaling.targetCpuValue` | — | Absolute CPU target for scale-up (e.g., `1500m`). Omit to disable CPU trigger. | +| `cluster..autoscaling.activationCpuValue` | — | CPU value below which CPU trigger is inactive. Required with targetCpuValue. | | `cluster..autoscaling.cooldownSeconds` | `300` | Seconds to wait after last scale event before scaling down again | | `cluster..autoscaling.gracePeriodSeconds` | `60-600` | Max time (seconds) to wait for graceful drain before forced termination | diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 6fc5916b84dd..0d12f5dd6248 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -48,6 +48,10 @@ spec: description: Autoscaling configuration (requires KEDA installed in the cluster) properties: + activationCpuValue: + description: CPU average value below which the trigger is + inactive. Required if targetCpuValue is set. + type: string cooldownSeconds: default: 600 description: Cooldown period in seconds after a scaling event @@ -70,17 +74,19 @@ spec: type: integer scaleDownThreshold: default: 20 - description: "Percentage threshold that triggers scale-down\ - \ (all conditions must be met: metric below threshold AND\ - \ CPU below threshold)" + description: Threshold that triggers scale-down for Prometheus-based + metrics type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ - \ with resources, pending tasks per AM for TezAM without\ - \ resources)" + \ sessions for HS2, connections for HMS, queue depth for\ + \ LLAP, pending tasks for TezAM)" type: integer + targetCpuValue: + description: "Target CPU average value for scaling (e.g.,\ + \ '1500m' or '1'). If omitted, CPU scaling is disabled." + type: string type: object configOverrides: additionalProperties: @@ -194,6 +200,10 @@ spec: description: Autoscaling configuration (requires KEDA installed in the cluster) properties: + activationCpuValue: + description: CPU average value below which the trigger is + inactive. Required if targetCpuValue is set. + type: string cooldownSeconds: default: 600 description: Cooldown period in seconds after a scaling event @@ -216,17 +226,19 @@ spec: type: integer scaleDownThreshold: default: 20 - description: "Percentage threshold that triggers scale-down\ - \ (all conditions must be met: metric below threshold AND\ - \ CPU below threshold)" + description: Threshold that triggers scale-down for Prometheus-based + metrics type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ - \ with resources, pending tasks per AM for TezAM without\ - \ resources)" + \ sessions for HS2, connections for HMS, queue depth for\ + \ LLAP, pending tasks for TezAM)" type: integer + targetCpuValue: + description: "Target CPU average value for scaling (e.g.,\ + \ '1500m' or '1'). If omitted, CPU scaling is disabled." + type: string type: object configOverrides: additionalProperties: @@ -315,6 +327,10 @@ spec: description: Autoscaling configuration (requires KEDA installed in the cluster) properties: + activationCpuValue: + description: CPU average value below which the trigger is + inactive. Required if targetCpuValue is set. + type: string cooldownSeconds: default: 600 description: Cooldown period in seconds after a scaling event @@ -337,17 +353,19 @@ spec: type: integer scaleDownThreshold: default: 20 - description: "Percentage threshold that triggers scale-down\ - \ (all conditions must be met: metric below threshold AND\ - \ CPU below threshold)" + description: Threshold that triggers scale-down for Prometheus-based + metrics type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ - \ with resources, pending tasks per AM for TezAM without\ - \ resources)" + \ sessions for HS2, connections for HMS, queue depth for\ + \ LLAP, pending tasks for TezAM)" type: integer + targetCpuValue: + description: "Target CPU average value for scaling (e.g.,\ + \ '1500m' or '1'). If omitted, CPU scaling is disabled." + type: string type: object configOverrides: additionalProperties: @@ -489,6 +507,10 @@ spec: description: Autoscaling configuration (requires KEDA installed in the cluster) properties: + activationCpuValue: + description: CPU average value below which the trigger is + inactive. Required if targetCpuValue is set. + type: string cooldownSeconds: default: 600 description: Cooldown period in seconds after a scaling event @@ -511,17 +533,19 @@ spec: type: integer scaleDownThreshold: default: 20 - description: "Percentage threshold that triggers scale-down\ - \ (all conditions must be met: metric below threshold AND\ - \ CPU below threshold)" + description: Threshold that triggers scale-down for Prometheus-based + metrics type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, queue depth for LLAP, CPU% for TezAM\ - \ with resources, pending tasks per AM for TezAM without\ - \ resources)" + \ sessions for HS2, connections for HMS, queue depth for\ + \ LLAP, pending tasks for TezAM)" type: integer + targetCpuValue: + description: "Target CPU average value for scaling (e.g.,\ + \ '1500m' or '1'). If omitted, CPU scaling is disabled." + type: string type: object configOverrides: additionalProperties: diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index c490b1f5d333..cbfe053a0e48 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -73,6 +73,12 @@ spec: minReplicas: {{ .Values.cluster.metastore.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.metastore.autoscaling.scaleUpThreshold }} scaleDownThreshold: {{ .Values.cluster.metastore.autoscaling.scaleDownThreshold }} + {{- if .Values.cluster.metastore.autoscaling.targetCpuValue }} + targetCpuValue: {{ .Values.cluster.metastore.autoscaling.targetCpuValue | quote }} + {{- end }} + {{- if .Values.cluster.metastore.autoscaling.activationCpuValue }} + activationCpuValue: {{ .Values.cluster.metastore.autoscaling.activationCpuValue | quote }} + {{- end }} cooldownSeconds: {{ .Values.cluster.metastore.autoscaling.cooldownSeconds }} gracePeriodSeconds: {{ .Values.cluster.metastore.autoscaling.gracePeriodSeconds }} {{- end }} @@ -111,6 +117,12 @@ spec: minReplicas: {{ .Values.cluster.hiveServer2.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleUpThreshold }} scaleDownThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownThreshold }} + {{- if .Values.cluster.hiveServer2.autoscaling.targetCpuValue }} + targetCpuValue: {{ .Values.cluster.hiveServer2.autoscaling.targetCpuValue | quote }} + {{- end }} + {{- if .Values.cluster.hiveServer2.autoscaling.activationCpuValue }} + activationCpuValue: {{ .Values.cluster.hiveServer2.autoscaling.activationCpuValue | quote }} + {{- end }} cooldownSeconds: {{ .Values.cluster.hiveServer2.autoscaling.cooldownSeconds }} gracePeriodSeconds: {{ .Values.cluster.hiveServer2.autoscaling.gracePeriodSeconds }} {{- end }} @@ -179,6 +191,12 @@ spec: minReplicas: {{ .Values.cluster.tezAm.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleUpThreshold }} scaleDownThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleDownThreshold }} + {{- if .Values.cluster.tezAm.autoscaling.targetCpuValue }} + targetCpuValue: {{ .Values.cluster.tezAm.autoscaling.targetCpuValue | quote }} + {{- end }} + {{- if .Values.cluster.tezAm.autoscaling.activationCpuValue }} + activationCpuValue: {{ .Values.cluster.tezAm.autoscaling.activationCpuValue | quote }} + {{- end }} cooldownSeconds: {{ .Values.cluster.tezAm.autoscaling.cooldownSeconds }} gracePeriodSeconds: {{ .Values.cluster.tezAm.autoscaling.gracePeriodSeconds }} {{- end }} diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index a0823f90de0d..a16a17e320e6 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -119,6 +119,8 @@ cluster: minReplicas: 1 scaleUpThreshold: 75 scaleDownThreshold: 30 + # targetCpuValue: "750m" # Uncomment to enable CPU-based scaling (AverageValue) + # activationCpuValue: "200m" # CPU trigger inactive below this value cooldownSeconds: 300 gracePeriodSeconds: 60 # Set to use an external Metastore instead of deploying one: @@ -144,6 +146,8 @@ cluster: minReplicas: 0 scaleUpThreshold: 80 scaleDownThreshold: 20 + # targetCpuValue: "1600m" # Uncomment to enable CPU-based scaling (AverageValue) + # activationCpuValue: "400m" # CPU trigger inactive below this value cooldownSeconds: 600 gracePeriodSeconds: 300 @@ -186,12 +190,13 @@ cluster: # Autoscaling (requires KEDA + Prometheus in the cluster) # minReplicas: 0 enables scale-to-zero — wakes when HS2 receives queries # When enabled, 'replicas' above acts as the max replica ceiling - # scaleUpThreshold: with CPU resources set → CPU% (e.g., 60 = 60% utilization); - # without CPU resources → pending tasks per AM (e.g., 5 = scale when 5+ tasks waiting) + # scaleUpThreshold: pending tasks per AM (e.g., 5 = scale when 5+ tasks waiting) autoscaling: enabled: false minReplicas: 0 scaleUpThreshold: 5 scaleDownThreshold: 10 + # targetCpuValue: "600m" # Uncomment to enable CPU-based scaling (AverageValue) + # activationCpuValue: "100m" # CPU trigger inactive below this value cooldownSeconds: 600 gracePeriodSeconds: 120 diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java index f5efb1302bd3..9e9c782e729e 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java @@ -30,6 +30,8 @@ import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.util.Labels; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Unified KEDA ScaledObject dependent resource for metric-based autoscaling. @@ -41,6 +43,8 @@ */ public abstract class HiveScaledObjectDependent extends HiveGenericDependentResource { + private static final Logger LOG = LoggerFactory.getLogger(HiveScaledObjectDependent.class); + private final String component; private final String targetKind; @@ -113,7 +117,9 @@ protected abstract List> getTriggers( * HiveServer2 ScaledObject: scales on hs2_active_sessions + CPU. */ public static class HiveServer2 extends HiveScaledObjectDependent { - public HiveServer2() { super("hiveserver2", "Deployment"); } + public HiveServer2() { + super("hiveserver2", "Deployment"); + } @Override protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { @@ -153,15 +159,20 @@ protected List> getTriggers( "activationThreshold", "0" ) )); - if (hiveCluster.getSpec().hiveServer2().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", String.valueOf(autoscaling.scaleUpThreshold()), - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); + if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { + if (hiveCluster.getSpec().hiveServer2().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "AverageValue", + "metadata", Map.of( + "value", autoscaling.targetCpuValue(), + "activationValue", autoscaling.activationCpuValue() + ) + )); + } else { + LOG.warn("targetCpuValue is set for HiveServer2, but no pod resources are defined. " + + "Skipping CPU trigger to prevent erratic scaling."); + } } // When scale-to-zero is enabled, add KEDA HTTP Add-on external-push // trigger to wake HS2 from 0 when requests arrive at the interceptor. @@ -184,7 +195,9 @@ protected List> getTriggers( * Metastore ScaledObject: scales on open_connections + CPU. */ public static class Metastore extends HiveScaledObjectDependent { - public Metastore() { super("metastore", "Deployment"); } + public Metastore() { + super("metastore", "Deployment"); + } @Override protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { @@ -224,15 +237,20 @@ protected List> getTriggers( "activationThreshold", "0" ) )); - if (hiveCluster.getSpec().metastore().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", String.valueOf(autoscaling.scaleUpThreshold()), - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); + if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { + if (hiveCluster.getSpec().metastore().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "AverageValue", + "metadata", Map.of( + "value", autoscaling.targetCpuValue(), + "activationValue", autoscaling.activationCpuValue() + ) + )); + } else { + LOG.warn("targetCpuValue is set for Metastore, but no pod resources are defined. " + + "Skipping CPU trigger to prevent erratic scaling."); + } } return triggers; } @@ -243,7 +261,9 @@ protected List> getTriggers( * Scale-down is slow (preserves in-memory cache). */ public static class Llap extends HiveScaledObjectDependent { - public Llap() { super("llap", "StatefulSet"); } + public Llap() { + super("llap", "StatefulSet"); + } @Override protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { @@ -299,7 +319,9 @@ protected List> getTriggers( * Tez AMs run in a warm pool; claimed AMs consume CPU, idle ones do not. */ public static class TezAm extends HiveScaledObjectDependent { - public TezAm() { super("tezam", "StatefulSet"); } + public TezAm() { + super("tezam", "StatefulSet"); + } @Override protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { @@ -329,15 +351,20 @@ protected List> getTriggers( String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; String namespace = hiveCluster.getMetadata().getNamespace(); List> triggers = new ArrayList<>(); - if (hiveCluster.getSpec().tezAm().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "Utilization", - "metadata", Map.of( - "value", String.valueOf(autoscaling.scaleUpThreshold()), - "activationValue", String.valueOf(autoscaling.scaleDownThreshold()) - ) - )); + if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { + if (hiveCluster.getSpec().tezAm().resources() != null) { + triggers.add(Map.of( + "type", "cpu", + "metricType", "AverageValue", + "metadata", Map.of( + "value", autoscaling.targetCpuValue(), + "activationValue", autoscaling.activationCpuValue() + ) + )); + } else { + LOG.warn("targetCpuValue is set for TezAM, but no pod resources are defined. " + + "Skipping CPU trigger to prevent erratic scaling."); + } triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); } else { triggers.add(Map.of( diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java index f3a064f48b95..a191be2eceb8 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -25,7 +25,6 @@ import io.javaoperatorsdk.operator.api.config.informer.Informer; import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; @@ -74,7 +73,9 @@ protected Service desired(HiveCluster hiveCluster, + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class HiveServer2 extends HiveServiceDependent { - public HiveServer2() { super("hiveserver2"); } + public HiveServer2() { + super("hiveserver2"); + } @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { @@ -106,7 +107,9 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class Metastore extends HiveServiceDependent { - public Metastore() { super("metastore"); } + public Metastore() { + super("metastore"); + } @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { @@ -131,7 +134,9 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class Llap extends HiveServiceDependent { - public Llap() { super("llap"); } + public Llap() { + super("llap"); + } @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { @@ -153,7 +158,9 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class TezAm extends HiveServiceDependent { - public TezAm() { super("tezam"); } + public TezAm() { + super("tezam"); + } @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java index eb0980fb1a1d..388515a51eb8 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -31,14 +31,19 @@ public record AutoscalingSpec( @Default("0") Integer minReplicas, @JsonPropertyDescription("Threshold that triggers scale-up (component-specific: " - + "sessions for HS2, queue depth for LLAP, CPU% for TezAM with resources, " - + "pending tasks per AM for TezAM without resources)") + + "sessions for HS2, connections for HMS, queue depth for LLAP, " + + "pending tasks for TezAM)") @Default("80") Integer scaleUpThreshold, - @JsonPropertyDescription("Percentage threshold that triggers scale-down " - + "(all conditions must be met: metric below threshold AND CPU below threshold)") + @JsonPropertyDescription("Threshold that triggers scale-down for Prometheus-based metrics") @Default("20") Integer scaleDownThreshold, + @JsonPropertyDescription("Target CPU average value for scaling (e.g., '1500m' or '1'). " + + "If omitted, CPU scaling is disabled.") + String targetCpuValue, + @JsonPropertyDescription("CPU average value below which the trigger is inactive. " + + "Required if targetCpuValue is set.") + String activationCpuValue, @JsonPropertyDescription("Cooldown period in seconds after a scaling event before another can occur") @Default("600") Integer cooldownSeconds, diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index 993b452ba4b4..89c629ecdea2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -62,6 +62,6 @@ public record HiveServer2Spec( extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 80, 20, 600, 300); + false, 0, 80, 20, null, null, 600, 300); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index 34cfc872f189..7a1951407afc 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -68,7 +68,7 @@ public record LlapSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 1, 0, 900, 600); + false, 0, 1, 0, null, null, 900, 600); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 51dd6dea7259..f744b05b72f1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -69,7 +69,7 @@ public record MetastoreSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 75, 30, 300, 60); + false, 1, 75, 30, null, null, 300, 60); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index 6a0008b9147d..9827b7a7d733 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 5, 10, 600, 120); + false, 0, 5, 10, null, null, 600, 120); } public boolean isEnabled() { From 92e3312e4c25549ef74c3dd7284e131254aab513 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 28 May 2026 23:18:21 +0530 Subject: [PATCH 04/22] Fix HS2 Scaling Down --- .../operator/dependent/HiveScaledObjectDependent.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java index 9e9c782e729e..555a6bec9c82 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java @@ -67,12 +67,8 @@ protected GenericKubernetesResource desired(HiveCluster hiveCluster, "kind", targetKind, "name", targetName )); - int minReplicaCount = Math.max(1, autoscaling.minReplicas()); - spec.put("minReplicaCount", minReplicaCount); + spec.put("minReplicaCount", autoscaling.minReplicas()); spec.put("maxReplicaCount", maxReplicas); - if (autoscaling.minReplicas() == 0) { - spec.put("idleReplicaCount", 0); - } spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); spec.put("pollingInterval", getPollingInterval()); spec.put("advanced", getAdvanced(hiveCluster, autoscaling, maxReplicas)); From 926f1cfd2e237e30373b445de308bf0e088ebbb9 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 29 May 2026 18:28:53 +0530 Subject: [PATCH 05/22] Fixes --- packaging/src/kubernetes/README.md | 148 ++++++++---- .../crds/hiveclusters.hive.apache.org-v1.yml | 112 +++++++-- .../hive-operator/templates/hivecluster.yaml | 8 + .../kubernetes/helm/hive-operator/values.yaml | 8 + .../dependent/HiveDependentResource.java | 29 ++- .../HiveGenericDependentResource.java | 58 ++++- .../dependent/HiveScaledObjectDependent.java | 212 +++++++++--------- .../HiveServer2DeploymentDependent.java | 6 +- .../dependent/HiveServiceDependent.java | 1 + .../dependent/LlapStatefulSetDependent.java | 3 +- .../MetastoreDeploymentDependent.java | 17 +- .../dependent/TezAmStatefulSetDependent.java | 3 +- .../operator/model/spec/AutoscalingSpec.java | 28 ++- .../operator/model/spec/HiveServer2Spec.java | 2 +- .../operator/model/spec/LlapSpec.java | 2 +- .../operator/model/spec/MetastoreSpec.java | 2 +- .../operator/model/spec/TezAmSpec.java | 2 +- .../kubernetes/operator/util/ConfigUtils.java | 15 ++ .../operator/util/HiveConfigBuilder.java | 12 + 19 files changed, 467 insertions(+), 201 deletions(-) diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index afe55dbe9139..f52da4db6eb6 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -586,24 +586,57 @@ helm install prometheus prometheus-community/prometheus \ │ always running) │ │ 3. Kubernetes sends SIGTERM to selected pod │ │ 4. preStop hook runs: │ -│ - HS2: deregisters from ZK, drains open sessions │ -│ - HMS: sleeps 30s for in-flight Thrift RPCs │ -│ - LLAP: waits until all executors become idle │ -│ - TezAM: waits for current DAG completion │ -│ 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety net) │ -│ 6. Pod terminates only after drain completes │ +│ - HS2: deregisters from ZK, drains open sessions, kills JVM │ +│ - HMS: kills JVM (stateless HTTP — no drain needed) │ +│ - LLAP: waits until all executors become idle, kills JVM │ +│ - TezAM: waits for current DAG completion, kills JVM │ +│ 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety cap) │ +│ 6. Pod terminates immediately once drain completes (does NOT wait │ +│ the full grace period — it's only the upper safety bound) │ └─────────────────────────────────────────────────────────────────────┘ ``` +> **Note:** Shell entrypoints (PID 1) in containers don't forward SIGTERM to child +> processes. The preStop hook explicitly sends SIGTERM to the Hive/Tez Java process +> after drain completes, ensuring prompt shutdown without waiting for the grace period +> to expire. + +### Scaling Timers + +The autoscaling system uses four independent timing controls: + +| Timer | Config Field | Default | Purpose | +|-------|-------------|---------|---------| +| **Metrics scrape interval** | `metricsScrapeIntervalSeconds` | `10` | How often Prometheus scrapes the pod's metrics endpoint. This is the **biggest bottleneck** for autoscaling reaction time — KEDA cannot detect metric changes faster than the scrape interval. | +| **Scale-up stabilization** | `scaleUpStabilizationSeconds` | `60` | HPA window: picks the highest recommendation within this period before scaling up. Prevents flapping when metrics oscillate. Set to `0` for LLAP and TezAM (reactive dependents). | +| **Scale-down stabilization** | `scaleDownStabilizationSeconds` | `300` | HPA window: picks the highest recommendation within this period before scaling down. Prevents premature removal of pods during temporary load dips. | +| **KEDA cooldown** | `cooldownSeconds` | `300-900` | Time after **all** KEDA triggers go inactive (metric = 0) before KEDA scales from 1→0. Only relevant when `minReplicas: 0`. HPA handles N→1 transitions using the stabilization window. | + +**How they interact:** +- Load spike detected → Prometheus scrapes metric within `metricsScrapeIntervalSeconds` → HPA waits `scaleUpStabilizationSeconds` then scales up +- Load drops → HPA waits `scaleDownStabilizationSeconds` then scales down (N→1) +- All triggers inactive → KEDA waits `cooldownSeconds` then scales 1→0 + +**Tuning reaction time:** With defaults (`metricsScrapeIntervalSeconds: 10`, `scaleUpStabilizationSeconds: 0` for LLAP/TezAM), scale-up latency is ~15s (one scrape + KEDA polling). For HS2 with `scaleUpStabilizationSeconds: 60`, expect ~70s. Reducing `metricsScrapeIntervalSeconds` below 10 gives diminishing returns and increases Prometheus load. + ### Per-Component Scaling Logic -| Component | Scale-Up Trigger | Scale-Down Trigger | Cooldown | Native Metric | -|-----------|-----------------|-------------------|----------|---------------| -| **HiveServer2** | `hs2_open_sessions` > scaleUpThreshold **OR** CPU > 75% | Sessions below threshold **AND** CPU below scaleDownThreshold | 5 min | `hs2_open_sessions` | -| **Metastore** | `hive_metastore_open_connections` > scaleUpThreshold **OR** CPU > 75% | Connections below threshold **AND** CPU below scaleDownThreshold | 5 min | `hive_metastore_open_connections` | -| **LLAP** | Total busy slots > scaleUpThreshold (queued + busy executors) | All executors idle + no HS2 sessions | 15 min | `NumQueuedRequests`, `NumExecutorsConfigured`, `NumExecutorsAvailable` | -| **Tez AM** (with CPU resources) | Pod CPU > scaleUpThreshold% | Pod CPU < scaleDownThreshold% + no HS2 sessions | 5 min | Standard K8s CPU | -| **Tez AM** (without CPU resources) | `tez_session_pending_tasks` > scaleUpThreshold | No pending tasks + no HS2 sessions | 5 min | `tez_session_pending_tasks` | +| Component | Scale-Up Trigger | Scale-Down Trigger | Native Metric | +|-----------|-----------------|-------------------|---------------| +| **HiveServer2** | `sum(hs2_open_sessions)` > scaleUpThreshold **OR** CPU > targetCpuValue | Sessions below threshold **AND** CPU below activationCpuValue | `hs2_open_sessions` (sum across pods) | +| **Metastore** | API request rate > scaleUpThreshold **OR** CPU > targetCpuValue | Request rate below threshold **AND** CPU below activationCpuValue | `api_*_total` (manual delta for Prometheus 3.x compatibility) | +| **LLAP** | Total busy slots > scaleUpThreshold (queued + busy executors) | All executors idle + no HS2 sessions | `NumQueuedRequests`, `NumExecutorsConfigured`, `NumExecutorsAvailable` | +| **Tez AM** | max(`sum(hs2_open_sessions)`, `count(HS2 pods)` x `sessions.per.default.queue`) | All HS2 sessions closed | `hs2_open_sessions` (demand-driven, no CPU trigger) | + +**TezAM Scaling Model:** TezAM uses demand-driven scaling with two KEDA triggers (max wins): +1. **Session demand** — `sum(hs2_open_sessions)`: scales to match the total number of + concurrent sessions across all HS2 pods (each session needs its own exclusive TezAM). +2. **Pre-warm** — `count(HS2 pods) × hive.server2.tez.sessions.per.default.queue` (default 1): + ensures every HS2 pod has enough TezAM sessions pre-claimed from ZooKeeper before queries arrive. + +KEDA takes the maximum desired replicas across both triggers. This ensures TezAM capacity +is always sufficient for both current demand and eager session pre-warming. No CPU-based +trigger is used — TezAM scaling is purely demand-driven from HS2 metrics. ### Scale-to-Zero Architecture @@ -646,10 +679,18 @@ without creating a conflicting second ScaledObject. ``` The HS2 ScaledObject combines three trigger types in a single resource: -- **Prometheus trigger** (`hs2_open_sessions`) — session-aware scaling +- **Prometheus trigger** (`sum(hs2_open_sessions)`) — session-aware scaling using total + session count across all pods (`sum()` prevents premature scale-down of pods with + active sessions; `desired = ceil(sum / threshold)`) - **CPU trigger** (`AverageValue` in millicores) — load-based scaling when `targetCpuValue` is configured - **external-push trigger** — wake-from-zero via the KEDA HTTP Add-on interceptor +**Session protection:** The HS2 Service uses `sessionAffinity: ClientIP` to ensure +beeline clients always reach the same pod. The preStop hook deregisters the pod from +ZooKeeper (preventing new sessions) and waits for `hs2_open_sessions` to drain to 0 +before terminating. The `gracePeriodSeconds` (default 3600s) is a safety cap — the pod +terminates immediately once sessions drain, not after the full grace period. + The `InterceptorRoute` CRD (`http.keda.sh/v1beta1`) configures only the interceptor routing (host matching, backend target) without auto-creating a ScaledObject — this avoids the dual-HPA conflict that `HTTPScaledObject` would cause. @@ -668,7 +709,7 @@ avoids the dual-HPA conflict that `HTTPScaledObject` would cause. | **HS2** | 0 | `hs2_open_sessions = 0` for cooldown | HTTP request via KEDA interceptor (`external-push`) | | **HMS** | 1 | Never (always running) | N/A | | **LLAP** | 0 | `hs2_open_sessions = 0` for cooldown | `hs2_open_sessions > 0` (cross-component) | -| **TezAM** | 0 | `hs2_open_sessions = 0` + no pending tasks | `hs2_open_sessions > 0` (cross-component) | +| **TezAM** | 0 | No HS2 pods with open sessions | `hs2_open_sessions > 0` (cross-component, demand-driven) | ### Enabling Autoscaling @@ -739,16 +780,22 @@ cluster: enabled: true # minReplicas: 0 # default — scale to zero when idle (requires KEDA HTTP Add-on) # scaleUpThreshold: 80 # default — avg open sessions per pod triggering scale-up - # cooldownSeconds: 300 # default — 5 min before scaling down + # cooldownSeconds: 600 # default — KEDA 1→0 cooldown after all triggers inactive + # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window + # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window + # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) metastore: replicas: 6 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true # minReplicas: 0 # default — scale to zero when no connections - # scaleUpThreshold: 75 # default — total open connections triggering scale-up - # cooldownSeconds: 300 # default — 5 min cooldown + # scaleUpThreshold: 75 # default — API request rate (req/s) triggering scale-up + # cooldownSeconds: 300 # default — KEDA 1→0 cooldown after all triggers inactive + # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window + # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window # gracePeriodSeconds: 60 # default — fast drain (HMS is stateless) + # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval llap: replicas: 8 # Acts as maxReplicas when autoscaling is enabled @@ -756,16 +803,22 @@ cluster: enabled: true # minReplicas: 0 # default — scale to zero when no HS2 sessions # scaleUpThreshold: 1 # default — total busy slots (queued+running) triggering scale-up - # cooldownSeconds: 900 # default — 15 min (scaling down destroys in-memory cache) + # cooldownSeconds: 900 # default — KEDA 1→0 cooldown (scaling down destroys cache) + # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window + # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window # gracePeriodSeconds: 600 # default — 10 min drain for in-flight fragments + # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) tezAm: replicas: 10 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true # minReplicas: 0 # default — scale to zero when no HS2 sessions - # scaleUpThreshold: 5 # default — CPU% (with resources) or pending tasks (without) + # scaleUpThreshold: 1 # default — threshold for demand metric (1 = match HS2 pod count) + # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window + # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window # gracePeriodSeconds: 120 # default — 2 min drain for DAG completion + # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) ``` ```bash @@ -775,7 +828,7 @@ helm install hive ./helm/hive-operator -f values-autoscaling.yaml When autoscaling is enabled, the operator automatically: - Deploys the Prometheus JMX Exporter agent sidecar (port 9404, `/metrics`) - Enables `hive.server2.metrics.enabled` / `metastore.metrics.enabled` (JMX reporter) -- Adds Prometheus scrape annotations to pods +- Adds Prometheus scrape annotations to pods (including `prometheus.io/scrape-interval` for fast reaction) - Creates KEDA ScaledObjects with the configured thresholds - Creates PodDisruptionBudgets (minAvailable: 1) - Configures preStop lifecycle hooks for graceful drain @@ -786,14 +839,14 @@ When autoscaling is enabled, the operator automatically: | Component | Key Metrics | Purpose | |-----------|---------|---------| -| **HiveServer2** | `hs2_open_sessions`, `hs2_active_sessions`, `tez_session_pending_tasks` | Session count (scaling trigger), Tez AM demand | -| **Metastore** | `hive_metastore_open_connections`, `api_*_total` | Connection count (scaling trigger), API call rates | +| **HiveServer2** | `hs2_open_sessions` | Session count — used by HS2 ScaledObject (sum for scale-up protection) and TezAM ScaledObject (demand-driven scaling) | +| **Metastore** | `api_*_total` | API call counters (manual delta for Prometheus 3.x compatibility) | | **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available (scaling trigger) | -| **Tez AM** | Standard K8s CPU metrics or `tez_session_pending_tasks` (from HS2) | CPU utilization or pending task count (scaling trigger) | +| **Tez AM** | N/A (scales on HS2 metrics) | TezAM scaling is demand-driven from `hs2_open_sessions` — no TezAM-specific metrics needed | ### CPU-Based Scaling -The operator can include a **CPU trigger** in the ScaledObject for HS2, Metastore, and Tez AM. +The operator can include a **CPU trigger** in the ScaledObject for HS2 and Metastore. The trigger uses KEDA's `AverageValue` metric type with **absolute millicore targets** that you specify directly. This handles burstable QoS pods correctly — unlike `Utilization` (which measures against the CPU request), `AverageValue` uses actual CPU consumption in @@ -818,10 +871,6 @@ operator relies solely on the Prometheus-based trigger (sessions, connections, e **Example:** With `targetCpuValue: "1600m"` and `activationCpuValue: "400m"`, KEDA scales up when average pod CPU exceeds 1600m and considers the trigger inactive below 400m. -For Tez AM specifically, without CPU targets the operator uses `tez_session_pending_tasks` -(queued tasks waiting for AM slots) as the proportional scaler — this reflects real query -demand rather than connection count, avoiding spurious scale-ups from idle sessions. - To enable both Prometheus and CPU-based scaling: ```yaml @@ -833,7 +882,7 @@ cluster: requestsMemory: "2Gi" autoscaling: enabled: true - scaleUpThreshold: 1 # scale up when avg sessions > 1 per pod + scaleUpThreshold: 1 # scale up when total sessions > 1 targetCpuValue: "1600m" # scale up when avg CPU > 1600m per pod activationCpuValue: "400m" # CPU trigger inactive below 400m @@ -846,16 +895,6 @@ cluster: enabled: true targetCpuValue: "750m" activationCpuValue: "200m" - - tezAm: - resources: - requestsCpu: "250m" - limitsCpu: "1" - requestsMemory: "1Gi" - autoscaling: - enabled: true - targetCpuValue: "600m" - activationCpuValue: "100m" ``` | Setting | Effect on CPU trigger | @@ -864,9 +903,10 @@ cluster: | `activationCpuValue` | CPU below which trigger is inactive. **Required** with targetCpuValue. | | `resources` | Pod resources must be defined — operator warns and skips CPU trigger otherwise. | -> **Note:** LLAP scaling uses only Prometheus triggers (total busy slots) -> and does not include a CPU trigger, so LLAP does not require `targetCpuValue` -> for autoscaling to work. +> **Note:** LLAP and TezAM scaling use only Prometheus-based triggers and do not +> include CPU triggers. LLAP scales on total busy slots (queued + running executors). +> TezAM scales on demand — the number of active HS2 pods multiplied by +> `hive.server2.tez.sessions.per.default.queue` (default 1). ### Helm Values Reference (Autoscaling) @@ -877,8 +917,10 @@ cluster: | `cluster..autoscaling.minReplicas` | `0` (HS2/LLAP/TezAM), `1` (HMS) | Minimum replica count. Set to 0 for scale-to-zero | | `cluster..autoscaling.scaleUpThreshold` | varies | Metric threshold triggering scale-up | | `cluster..autoscaling.scaleDownThreshold` | varies | Metric threshold triggering scale-down | -| `cluster..autoscaling.cooldownSeconds` | varies | Cooldown after a scaling event | -| `cluster..autoscaling.gracePeriodSeconds` | varies | Max drain time before forced termination | +| `cluster..autoscaling.cooldownSeconds` | varies | KEDA cooldown: seconds after all triggers go inactive before scaling 1→0 | +| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | HPA stabilization window for scale-up (picks highest recommendation in window) | +| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300` | HPA stabilization window for scale-down (picks highest recommendation in window) | +| `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap: max drain time before forced termination. Pod exits immediately once sessions/connections drain to 0. | --- @@ -977,6 +1019,13 @@ kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hi > is the primary transport. To override, use `configOverrides` in the > HiveServer2 spec. +> **Metastore HTTP Mode:** The operator configures HMS in HTTP transport mode +> (`metastore.server.thrift.transport.mode=http`) and sets the matching client +> config (`hive.metastore.client.thrift.transport.mode=http`) on HS2 and TezAM. +> HTTP mode makes Metastore connections stateless — each RPC is an independent +> HTTP request, so Metastore pods can scale down safely without breaking active +> connections from HiveServer2. The port remains 9083 (same as binary mode). + --- ## Helm Values Reference @@ -1085,12 +1134,15 @@ kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hi |-------|---------|-------------| | `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling for this component | | `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (HS2 requires KEDA HTTP Add-on) | -| `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (sessions for HS2, connections for HMS, busy slots for LLAP, pending tasks or CPU% for TezAM) | +| `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (total sessions for HS2, request rate for HMS, busy slots for LLAP, demand per HS2 pod for TezAM) | | `cluster..autoscaling.scaleDownThreshold` | `30` | Prometheus metric threshold for scale-down (component-specific) | | `cluster..autoscaling.targetCpuValue` | — | Absolute CPU target for scale-up (e.g., `1500m`). Omit to disable CPU trigger. | | `cluster..autoscaling.activationCpuValue` | — | CPU value below which CPU trigger is inactive. Required with targetCpuValue. | -| `cluster..autoscaling.cooldownSeconds` | `300` | Seconds to wait after last scale event before scaling down again | -| `cluster..autoscaling.gracePeriodSeconds` | `60-600` | Max time (seconds) to wait for graceful drain before forced termination | +| `cluster..autoscaling.cooldownSeconds` | `300-900` | KEDA cooldown: seconds after all triggers go inactive before scaling 1→0 | +| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | HPA stabilization window for scale-up decisions (prevents flapping) | +| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300` | HPA stabilization window for scale-down decisions (prevents premature scale-down) | +| `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap (seconds) — pod terminates immediately once drain completes, this is only the upper bound | +| `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | Prometheus scrape interval override for this component's pods. Lower values make autoscaling react faster but increase Prometheus load. Applied via `prometheus.io/scrape-interval` pod annotation. | --- diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 0d12f5dd6248..1e496830f21b 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -54,17 +54,25 @@ spec: type: string cooldownSeconds: default: 600 - description: Cooldown period in seconds after a scaling event - before another can occur + description: Cooldown period in seconds after all KEDA triggers + are inactive before scaling from 1 to 0 (scale-to-zero delay) type: integer enabled: default: false description: Whether autoscaling is enabled for this component type: boolean gracePeriodSeconds: - default: 300 + default: 3600 description: Maximum time in seconds to wait for graceful - drain during scale-down before the pod is forcibly terminated + drain during scale-down before the pod is forcibly terminated. + The pod terminates immediately once sessions/connections + drain to 0; this value is only the upper safety cap. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: Prometheus scrape interval in seconds for this + component's metrics. Lower values make autoscaling react + faster but increase Prometheus load. type: integer minReplicas: default: 0 @@ -72,11 +80,23 @@ spec: Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero) type: integer + scaleDownStabilizationSeconds: + default: 300 + description: Stabilization window in seconds for scale-down + decisions. HPA picks the highest recommendation within this + window to prevent premature scale-down. + type: integer scaleDownThreshold: default: 20 description: Threshold that triggers scale-down for Prometheus-based metrics type: integer + scaleUpStabilizationSeconds: + default: 60 + description: Stabilization window in seconds for scale-up + decisions. HPA picks the highest recommendation within this + window to prevent flapping. + type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ @@ -206,17 +226,25 @@ spec: type: string cooldownSeconds: default: 600 - description: Cooldown period in seconds after a scaling event - before another can occur + description: Cooldown period in seconds after all KEDA triggers + are inactive before scaling from 1 to 0 (scale-to-zero delay) type: integer enabled: default: false description: Whether autoscaling is enabled for this component type: boolean gracePeriodSeconds: - default: 300 + default: 3600 description: Maximum time in seconds to wait for graceful - drain during scale-down before the pod is forcibly terminated + drain during scale-down before the pod is forcibly terminated. + The pod terminates immediately once sessions/connections + drain to 0; this value is only the upper safety cap. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: Prometheus scrape interval in seconds for this + component's metrics. Lower values make autoscaling react + faster but increase Prometheus load. type: integer minReplicas: default: 0 @@ -224,11 +252,23 @@ spec: Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero) type: integer + scaleDownStabilizationSeconds: + default: 300 + description: Stabilization window in seconds for scale-down + decisions. HPA picks the highest recommendation within this + window to prevent premature scale-down. + type: integer scaleDownThreshold: default: 20 description: Threshold that triggers scale-down for Prometheus-based metrics type: integer + scaleUpStabilizationSeconds: + default: 60 + description: Stabilization window in seconds for scale-up + decisions. HPA picks the highest recommendation within this + window to prevent flapping. + type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ @@ -333,17 +373,25 @@ spec: type: string cooldownSeconds: default: 600 - description: Cooldown period in seconds after a scaling event - before another can occur + description: Cooldown period in seconds after all KEDA triggers + are inactive before scaling from 1 to 0 (scale-to-zero delay) type: integer enabled: default: false description: Whether autoscaling is enabled for this component type: boolean gracePeriodSeconds: - default: 300 + default: 3600 description: Maximum time in seconds to wait for graceful - drain during scale-down before the pod is forcibly terminated + drain during scale-down before the pod is forcibly terminated. + The pod terminates immediately once sessions/connections + drain to 0; this value is only the upper safety cap. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: Prometheus scrape interval in seconds for this + component's metrics. Lower values make autoscaling react + faster but increase Prometheus load. type: integer minReplicas: default: 0 @@ -351,11 +399,23 @@ spec: Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero) type: integer + scaleDownStabilizationSeconds: + default: 300 + description: Stabilization window in seconds for scale-down + decisions. HPA picks the highest recommendation within this + window to prevent premature scale-down. + type: integer scaleDownThreshold: default: 20 description: Threshold that triggers scale-down for Prometheus-based metrics type: integer + scaleUpStabilizationSeconds: + default: 60 + description: Stabilization window in seconds for scale-up + decisions. HPA picks the highest recommendation within this + window to prevent flapping. + type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ @@ -513,17 +573,25 @@ spec: type: string cooldownSeconds: default: 600 - description: Cooldown period in seconds after a scaling event - before another can occur + description: Cooldown period in seconds after all KEDA triggers + are inactive before scaling from 1 to 0 (scale-to-zero delay) type: integer enabled: default: false description: Whether autoscaling is enabled for this component type: boolean gracePeriodSeconds: - default: 300 + default: 3600 description: Maximum time in seconds to wait for graceful - drain during scale-down before the pod is forcibly terminated + drain during scale-down before the pod is forcibly terminated. + The pod terminates immediately once sessions/connections + drain to 0; this value is only the upper safety cap. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: Prometheus scrape interval in seconds for this + component's metrics. Lower values make autoscaling react + faster but increase Prometheus load. type: integer minReplicas: default: 0 @@ -531,11 +599,23 @@ spec: Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero) type: integer + scaleDownStabilizationSeconds: + default: 300 + description: Stabilization window in seconds for scale-down + decisions. HPA picks the highest recommendation within this + window to prevent premature scale-down. + type: integer scaleDownThreshold: default: 20 description: Threshold that triggers scale-down for Prometheus-based metrics type: integer + scaleUpStabilizationSeconds: + default: 60 + description: Stabilization window in seconds for scale-up + decisions. HPA picks the highest recommendation within this + window to prevent flapping. + type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index cbfe053a0e48..9ed5269db04c 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -80,6 +80,8 @@ spec: activationCpuValue: {{ .Values.cluster.metastore.autoscaling.activationCpuValue | quote }} {{- end }} cooldownSeconds: {{ .Values.cluster.metastore.autoscaling.cooldownSeconds }} + scaleUpStabilizationSeconds: {{ .Values.cluster.metastore.autoscaling.scaleUpStabilizationSeconds }} + scaleDownStabilizationSeconds: {{ .Values.cluster.metastore.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.metastore.autoscaling.gracePeriodSeconds }} {{- end }} {{- else }} @@ -124,6 +126,8 @@ spec: activationCpuValue: {{ .Values.cluster.hiveServer2.autoscaling.activationCpuValue | quote }} {{- end }} cooldownSeconds: {{ .Values.cluster.hiveServer2.autoscaling.cooldownSeconds }} + scaleUpStabilizationSeconds: {{ .Values.cluster.hiveServer2.autoscaling.scaleUpStabilizationSeconds }} + scaleDownStabilizationSeconds: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.hiveServer2.autoscaling.gracePeriodSeconds }} {{- end }} @@ -157,6 +161,8 @@ spec: scaleUpThreshold: {{ .Values.cluster.llap.autoscaling.scaleUpThreshold }} scaleDownThreshold: {{ .Values.cluster.llap.autoscaling.scaleDownThreshold }} cooldownSeconds: {{ .Values.cluster.llap.autoscaling.cooldownSeconds }} + scaleUpStabilizationSeconds: {{ .Values.cluster.llap.autoscaling.scaleUpStabilizationSeconds }} + scaleDownStabilizationSeconds: {{ .Values.cluster.llap.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.llap.autoscaling.gracePeriodSeconds }} {{- end }} {{- end }} @@ -198,6 +204,8 @@ spec: activationCpuValue: {{ .Values.cluster.tezAm.autoscaling.activationCpuValue | quote }} {{- end }} cooldownSeconds: {{ .Values.cluster.tezAm.autoscaling.cooldownSeconds }} + scaleUpStabilizationSeconds: {{ .Values.cluster.tezAm.autoscaling.scaleUpStabilizationSeconds }} + scaleDownStabilizationSeconds: {{ .Values.cluster.tezAm.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.tezAm.autoscaling.gracePeriodSeconds }} {{- end }} {{- end }} diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index a16a17e320e6..85bb02c1277f 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -122,6 +122,8 @@ cluster: # targetCpuValue: "750m" # Uncomment to enable CPU-based scaling (AverageValue) # activationCpuValue: "200m" # CPU trigger inactive below this value cooldownSeconds: 300 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 60 # Set to use an external Metastore instead of deploying one: # enabled: false @@ -149,6 +151,8 @@ cluster: # targetCpuValue: "1600m" # Uncomment to enable CPU-based scaling (AverageValue) # activationCpuValue: "400m" # CPU trigger inactive below this value cooldownSeconds: 600 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 300 # --------------------------------------------------------------------------- @@ -173,6 +177,8 @@ cluster: scaleUpThreshold: 1 scaleDownThreshold: 0 cooldownSeconds: 900 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 600 # --------------------------------------------------------------------------- @@ -199,4 +205,6 @@ cluster: # targetCpuValue: "600m" # Uncomment to enable CPU-based scaling (AverageValue) # activationCpuValue: "100m" # CPU trigger inactive below this value cooldownSeconds: 600 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 120 diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 7fc1290d5375..3a47e4e114b4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -242,6 +242,13 @@ protected static String buildDrainScript( lines.add(" RETRIES=0"); lines.add(" sleep " + sleepSeconds); lines.add("done"); + // Send SIGTERM directly to the Java process. Shell entrypoint scripts + // (PID 1) often don't forward signals, so K8s SIGTERM never reaches + // the JVM — causing a full grace-period wait before SIGKILL. + // Use 'java' pattern to avoid matching this script itself. + lines.add("echo '[preStop] Sending SIGTERM to Java process...'"); + lines.add("kill $(pgrep -f 'java.*org.apache') 2>/dev/null"); + lines.add("exit 0"); return String.join("\n", lines); } @@ -299,6 +306,13 @@ protected static String buildDualMetricDrainScript( lines.add(" RETRIES=0"); lines.add(" sleep " + sleepSeconds); lines.add("done"); + // Send SIGTERM directly to the Java process. Shell entrypoint scripts + // (PID 1) often don't forward signals, so K8s SIGTERM never reaches + // the JVM — causing a full grace-period wait before SIGKILL. + // Use 'java' pattern to avoid matching this script itself. + lines.add("echo '[preStop] Sending SIGTERM to Java process...'"); + lines.add("kill $(pgrep -f 'java.*org.apache') 2>/dev/null"); + lines.add("exit 0"); return String.join("\n", lines); } @@ -611,7 +625,8 @@ protected static Probe buildTcpProbe(int port, ProbeSpec spec, int defaultInitia protected static void applyAutoscalingLifecycle( io.fabric8.kubernetes.api.model.PodSpec podSpec, io.fabric8.kubernetes.api.model.ObjectMeta podMetadata, - String preStopScript, int gracePeriodSeconds) { + String preStopScript, int gracePeriodSeconds, + int metricsScrapeIntervalSeconds) { io.fabric8.kubernetes.api.model.Lifecycle lifecycle = new io.fabric8.kubernetes.api.model.LifecycleBuilder() .withNewPreStop() @@ -622,10 +637,22 @@ protected static void applyAutoscalingLifecycle( .build(); podSpec.getContainers().get(0).setLifecycle(lifecycle); podSpec.setTerminationGracePeriodSeconds((long) gracePeriodSeconds); + applyPrometheusScrapeAnnotations(podMetadata, metricsScrapeIntervalSeconds); + } + + /** + * Adds Prometheus scrape annotations to a pod template so that + * the JMX Exporter metrics endpoint is discovered by Prometheus. + */ + private static void applyPrometheusScrapeAnnotations( + io.fabric8.kubernetes.api.model.ObjectMeta podMetadata, + int scrapeIntervalSeconds) { podMetadata.getAnnotations().put("prometheus.io/scrape", "true"); podMetadata.getAnnotations().put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); podMetadata.getAnnotations().put("prometheus.io/path", "/metrics"); + podMetadata.getAnnotations().put("prometheus.io/scrape-interval", + scrapeIntervalSeconds + "s"); } /** diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java index 6db1482f3bd1..de9fb6351824 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java @@ -24,6 +24,7 @@ import java.util.Optional; import java.util.Set; + import io.fabric8.kubernetes.api.model.GenericKubernetesResource; import io.javaoperatorsdk.operator.api.config.informer.InformerEventSourceConfiguration; import io.javaoperatorsdk.operator.api.reconciler.Context; @@ -145,17 +146,64 @@ protected static Map buildHpaBehavior( */ protected static Map buildHs2ActivationTrigger( String namespace, String hs2TargetName, int maxReplicas) { + return buildPrometheusTrigger( + "hs2_open_sessions_activation", + String.format( + "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", + namespace, hs2TargetName), + String.valueOf(maxReplicas)); + } + + /** + * Builds a KEDA Prometheus trigger entry. + * + * @param metricName the KEDA metric name + * @param query the PromQL query + * @param threshold the scaling threshold value + */ + protected static Map buildPrometheusTrigger( + String metricName, String query, String threshold) { return Map.of( "type", "prometheus", "metadata", Map.of( "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_open_sessions_activation", - "query", String.format( - "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(maxReplicas), + "metricName", metricName, + "query", query, + "threshold", threshold, "activationThreshold", "0" ) ); } + + /** + * Builds a KEDA CPU AverageValue trigger if both targetCpuValue and + * activationCpuValue are configured. Returns null if CPU scaling is + * not configured, or if resources are missing (logs a warning). + * + * @param autoscaling the autoscaling spec + * @param resources the pod resource spec (null means not set) + * @param componentName component name for the warning message + * @param log the logger to use for warnings + */ + protected static Map buildCpuTrigger( + org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec autoscaling, + Object resources, String componentName, + org.slf4j.Logger log) { + if (autoscaling.targetCpuValue() == null || autoscaling.activationCpuValue() == null) { + return null; + } + if (resources == null) { + log.warn("targetCpuValue is set for {}, but no pod resources are defined. " + + "Skipping CPU trigger to prevent erratic scaling.", componentName); + return null; + } + return Map.of( + "type", "cpu", + "metricType", "AverageValue", + "metadata", Map.of( + "value", autoscaling.targetCpuValue(), + "activationValue", autoscaling.activationCpuValue() + ) + ); + } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java index 555a6bec9c82..52a3624639ca 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java @@ -29,6 +29,7 @@ import io.javaoperatorsdk.operator.processing.GroupVersionKind; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -128,14 +129,16 @@ protected int getMaxReplicas(HiveCluster hiveCluster) { } @Override - protected int getPollingInterval() { return 30; } + protected int getPollingInterval() { + return 30; + } @Override protected Map getAdvanced( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { return buildHpaBehavior( - autoscaling.cooldownSeconds(), "Pods", 1, 60, - 60, "Percent", 100, 60); + autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, + autoscaling.scaleUpStabilizationSeconds(), "Percent", 100, 60); } @Override @@ -143,32 +146,22 @@ protected List> getTriggers( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas, String targetName) { List> triggers = new ArrayList<>(); - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hs2_open_sessions", - "query", String.format( - "avg(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - )); - if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { - if (hiveCluster.getSpec().hiveServer2().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "AverageValue", - "metadata", Map.of( - "value", autoscaling.targetCpuValue(), - "activationValue", autoscaling.activationCpuValue() - ) - )); - } else { - LOG.warn("targetCpuValue is set for HiveServer2, but no pod resources are defined. " - + "Skipping CPU trigger to prevent erratic scaling."); - } + // Use sum() so KEDA computes desired replicas from total session count. + // desired = ceil(sum / threshold). With sum=2, threshold=1: desired=2 + // → prevents premature scale-down while sessions are active. + // avg() would divide across pods, hiding load and causing scale-down + // of pods with sessions. + triggers.add(buildPrometheusTrigger( + "hs2_open_sessions", + String.format( + "sum(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName), + String.valueOf(autoscaling.scaleUpThreshold()))); + Map cpuTrigger = buildCpuTrigger( + autoscaling, hiveCluster.getSpec().hiveServer2().resources(), + "HiveServer2", LOG); + if (cpuTrigger != null) { + triggers.add(cpuTrigger); } // When scale-to-zero is enabled, add KEDA HTTP Add-on external-push // trigger to wake HS2 from 0 when requests arrive at the interceptor. @@ -206,14 +199,16 @@ protected int getMaxReplicas(HiveCluster hiveCluster) { } @Override - protected int getPollingInterval() { return 30; } + protected int getPollingInterval() { + return 30; + } @Override protected Map getAdvanced( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { return buildHpaBehavior( - autoscaling.cooldownSeconds(), "Pods", 1, 60, - 120, "Percent", 50, 60); + autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, + autoscaling.scaleUpStabilizationSeconds(), "Percent", 50, 60); } @Override @@ -221,32 +216,24 @@ protected List> getTriggers( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas, String targetName) { List> triggers = new ArrayList<>(); - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "hive_metastore_open_connections", - "query", String.format( - "sum(hive_metastore_open_connections{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - )); - if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { - if (hiveCluster.getSpec().metastore().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "AverageValue", - "metadata", Map.of( - "value", autoscaling.targetCpuValue(), - "activationValue", autoscaling.activationCpuValue() - ) - )); - } else { - LOG.warn("targetCpuValue is set for Metastore, but no pod resources are defined. " - + "Skipping CPU trigger to prevent erratic scaling."); - } + // HMS runs in HTTP transport mode — connections are per-request (stateless), + // so open_connections is always ~0. Use aggregate API request rate instead. + // Note: Prometheus 3.x rejects rate() on __name__ regex selectors, so we + // compute rate manually as (sum(counters) - sum(counters offset 2m)) / 120. + triggers.add(buildPrometheusTrigger( + "hive_metastore_api_rate", + String.format( + "(sum({__name__=~\"api_.+_total\",namespace=\"%s\",pod=~\"%s-.*\"})" + + " - sum({__name__=~\"api_.+_total\",namespace=\"%s\",pod=~\"%s-.*\"} offset 2m))" + + " / 120 or vector(0)", + hiveCluster.getMetadata().getNamespace(), targetName, + hiveCluster.getMetadata().getNamespace(), targetName), + String.valueOf(autoscaling.scaleUpThreshold()))); + Map cpuTrigger = buildCpuTrigger( + autoscaling, hiveCluster.getSpec().metastore().resources(), + "Metastore", LOG); + if (cpuTrigger != null) { + triggers.add(cpuTrigger); } return triggers; } @@ -272,13 +259,17 @@ protected int getMaxReplicas(HiveCluster hiveCluster) { } @Override - protected int getPollingInterval() { return 5; } + protected int getPollingInterval() { + return 5; + } @Override protected Map getAdvanced( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + // Scale-up stabilization=0: LLAP is a reactive dependent that must + // track HS2 immediately — no delay on scale-up. return buildHpaBehavior( - autoscaling.cooldownSeconds(), "Pods", 1, autoscaling.cooldownSeconds(), + autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, autoscaling.scaleDownStabilizationSeconds(), 0, "Pods", maxReplicas, 15); } @@ -289,30 +280,26 @@ protected List> getTriggers( String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; String namespace = hiveCluster.getMetadata().getNamespace(); return List.of( - Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "llap_total_busy_slots", - "query", String.format( - "avg(" - + "hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + " + on(pod) hadoop_llapdaemon_executornumexecutorsconfigured{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + " - on(pod) hadoop_llapdaemon_executornumexecutorsavailable{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + ") or vector(0)", - namespace, targetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - ), + buildPrometheusTrigger( + "llap_total_busy_slots", + String.format( + "avg(" + + "hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + " + on(pod) hadoop_llapdaemon_executornumexecutorsconfigured{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + " - on(pod) hadoop_llapdaemon_executornumexecutorsavailable{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" + + ") or vector(0)", + namespace, targetName), + String.valueOf(autoscaling.scaleUpThreshold())), buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas) ); } } /** - * TezAM ScaledObject: scales on CPU (or pending tasks) + HS2 activation trigger. - * Tez AMs run in a warm pool; claimed AMs consume CPU, idle ones do not. + * TezAM ScaledObject: scales on HS2 session demand. + * Each HS2 pod claims {@code sessions.per.default.queue} TezAM sessions + * (exclusive binding). Demand = active HS2 pods × sessions per queue. + * Primary trigger: count of HS2 pods with open sessions × sessions_per_queue. */ public static class TezAm extends HiveScaledObjectDependent { public TezAm() { @@ -330,14 +317,18 @@ protected int getMaxReplicas(HiveCluster hiveCluster) { } @Override - protected int getPollingInterval() { return 5; } + protected int getPollingInterval() { + return 5; + } @Override protected Map getAdvanced( HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { + // Scale-up stabilization=0: TezAM is a reactive dependent that must + // track HS2 sessions immediately — no delay on scale-up. return buildHpaBehavior( - autoscaling.cooldownSeconds(), "Pods", 1, 60, - 60, "Pods", 2, 30); + autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, + 0, "Pods", maxReplicas, 15); } @Override @@ -346,37 +337,36 @@ protected List> getTriggers( int maxReplicas, String targetName) { String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; String namespace = hiveCluster.getMetadata().getNamespace(); + + // Read sessions.per.default.queue from HS2 configOverrides (default 1). + // Each HS2 pod pre-warms this many TezAM sessions in its pool. + int sessionsPerQueue = ConfigUtils.getInt( + hiveCluster.getSpec().hiveServer2().configOverrides(), + ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY, + null, ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT); + List> triggers = new ArrayList<>(); - if (autoscaling.targetCpuValue() != null && autoscaling.activationCpuValue() != null) { - if (hiveCluster.getSpec().tezAm().resources() != null) { - triggers.add(Map.of( - "type", "cpu", - "metricType", "AverageValue", - "metadata", Map.of( - "value", autoscaling.targetCpuValue(), - "activationValue", autoscaling.activationCpuValue() - ) - )); - } else { - LOG.warn("targetCpuValue is set for TezAM, but no pod resources are defined. " - + "Skipping CPU trigger to prevent erratic scaling."); - } - triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); - } else { - triggers.add(Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", "tez_session_pending_tasks", - "query", String.format( - "sum(tez_session_pending_tasks{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - namespace, hs2TargetName), - "threshold", String.valueOf(autoscaling.scaleUpThreshold()), - "activationThreshold", "0" - ) - )); - triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); - } + // Trigger 1: Concurrent demand — total open sessions across all HS2 pods. + // Each session may run a query needing its own TezAM. + // threshold=1 → desired = total open sessions. + triggers.add(buildPrometheusTrigger( + "hs2_tezam_session_demand", + String.format( + "sum(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", + namespace, hs2TargetName), + "1")); + // Trigger 2: Pre-warm — each running HS2 pod needs sessions_per_queue TezAMs + // in its pool (claimed eagerly at startup by default). + // threshold=1 → desired = HS2_pod_count × sessions_per_queue. + triggers.add(buildPrometheusTrigger( + "hs2_tezam_prewarm", + String.format( + "count(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) * %d or vector(0)", + namespace, hs2TargetName, sessionsPerQueue), + "1")); + // KEDA uses max(trigger1, trigger2) → ensures enough TezAMs for both + // concurrent queries AND per-HS2 pre-warm pools. + triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); return triggers; } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index c08796c218a7..20d27a46b0f7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -265,7 +265,8 @@ protected Deployment desired(HiveCluster hiveCluster, if (autoscaling.isEnabled()) { List zkDeregister = List.of( "echo '[preStop] Deregistering HiveServer2 from ZooKeeper...'", - "hive --service hiveserver2 --deregister || echo '[preStop] WARNING: ZK deregister failed'"); + "hive --service hiveserver2 --deregister $(hive --service version 2>/dev/null | head -1 || echo '4.0.0')" + + " || echo '[preStop] WARNING: ZK deregister failed'"); String preStopScript = buildDrainScript( "Waiting for open sessions to drain", "hs2_open_sessions", "SESSIONS", @@ -274,7 +275,8 @@ protected Deployment desired(HiveCluster hiveCluster, applyAutoscalingLifecycle( deployment.getSpec().getTemplate().getSpec(), deployment.getSpec().getTemplate().getMetadata(), - preStopScript, autoscaling.gracePeriodSeconds()); + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } appendUserVolumes(deployment.getSpec().getTemplate().getSpec(), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java index a191be2eceb8..edd048e8a322 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -91,6 +91,7 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { null, ConfigUtils.HIVE_SERVER2_WEBUI_PORT_DEFAULT); builder.editSpec() .withType(hs2.serviceType()) + .withSessionAffinity("ClientIP") .addNewPort().withName("thrift").withProtocol("TCP") .withPort(thriftPort).withTargetPort(new IntOrString(thriftPort)).endPort() .addNewPort().withName("http").withProtocol("TCP") diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 7bd1473afefc..171766e6f341 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -192,7 +192,8 @@ protected StatefulSet desired(HiveCluster hiveCluster, applyAutoscalingLifecycle( statefulSet.getSpec().getTemplate().getSpec(), statefulSet.getSpec().getTemplate().getMetadata(), - preStopScript, autoscaling.gracePeriodSeconds()); + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index ce16cc17eeaf..4b39d0341a51 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -175,17 +175,20 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); - // Graceful scale-down: poll JMX Exporter (port 9404) for open_connections to drain. + // HMS uses HTTP transport mode — connections are stateless, so no session + // drain is needed. The preStop hook simply sends SIGTERM directly to the + // JVM (the shell entrypoint doesn't forward signals from K8s). if (autoscaling.isEnabled()) { - String preStopScript = buildDrainScript( - "Waiting for open connections to drain", - "hive_metastore_open_connections", "CONNS", - "All connections drained. Shutting down.", - 5, 6, null); + String preStopScript = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Sending SIGTERM to Metastore Java process...'", + "kill $(pgrep -f 'java.*org.apache') 2>/dev/null", + "exit 0"); applyAutoscalingLifecycle( deployment.getSpec().getTemplate().getSpec(), deployment.getSpec().getTemplate().getMetadata(), - preStopScript, autoscaling.gracePeriodSeconds()); + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } appendUserVolumes(deployment.getSpec().getTemplate().getSpec(), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index 01a0509f99ca..ac83286e346c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -191,7 +191,8 @@ protected StatefulSet desired(HiveCluster hiveCluster, applyAutoscalingLifecycle( statefulSet.getSpec().getTemplate().getSpec(), statefulSet.getSpec().getTemplate().getMetadata(), - preStopScript, autoscaling.gracePeriodSeconds()); + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java index 388515a51eb8..ab02949d7f25 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -44,13 +44,28 @@ public record AutoscalingSpec( @JsonPropertyDescription("CPU average value below which the trigger is inactive. " + "Required if targetCpuValue is set.") String activationCpuValue, - @JsonPropertyDescription("Cooldown period in seconds after a scaling event before another can occur") + @JsonPropertyDescription("Cooldown period in seconds after all KEDA triggers are inactive " + + "before scaling from 1 to 0 (scale-to-zero delay)") @Default("600") Integer cooldownSeconds, - @JsonPropertyDescription("Maximum time in seconds to wait for graceful drain " - + "during scale-down before the pod is forcibly terminated") + @JsonPropertyDescription("Stabilization window in seconds for scale-up decisions. " + + "HPA picks the highest recommendation within this window to prevent flapping.") + @Default("60") + Integer scaleUpStabilizationSeconds, + @JsonPropertyDescription("Stabilization window in seconds for scale-down decisions. " + + "HPA picks the highest recommendation within this window to prevent premature scale-down.") @Default("300") - Integer gracePeriodSeconds) { + Integer scaleDownStabilizationSeconds, + @JsonPropertyDescription("Maximum time in seconds to wait for graceful drain " + + "during scale-down before the pod is forcibly terminated. " + + "The pod terminates immediately once sessions/connections drain to 0; " + + "this value is only the upper safety cap.") + @Default("3600") + Integer gracePeriodSeconds, + @JsonPropertyDescription("Prometheus scrape interval in seconds for this component's metrics. " + + "Lower values make autoscaling react faster but increase Prometheus load.") + @Default("10") + Integer metricsScrapeIntervalSeconds) { public AutoscalingSpec { enabled = enabled != null ? enabled : false; @@ -58,7 +73,10 @@ public record AutoscalingSpec( scaleUpThreshold = scaleUpThreshold != null ? scaleUpThreshold : 80; scaleDownThreshold = scaleDownThreshold != null ? scaleDownThreshold : 20; cooldownSeconds = cooldownSeconds != null ? cooldownSeconds : 600; - gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 300; + scaleUpStabilizationSeconds = scaleUpStabilizationSeconds != null ? scaleUpStabilizationSeconds : 60; + scaleDownStabilizationSeconds = scaleDownStabilizationSeconds != null ? scaleDownStabilizationSeconds : 300; + gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 3600; + metricsScrapeIntervalSeconds = metricsScrapeIntervalSeconds != null ? metricsScrapeIntervalSeconds : 10; } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index 89c629ecdea2..b81ad83b41b7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -62,6 +62,6 @@ public record HiveServer2Spec( extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 80, 20, null, null, 600, 300); + false, 0, 80, 20, null, null, 600, 60, 300, 300, 10); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index 7a1951407afc..c24bac5a1116 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -68,7 +68,7 @@ public record LlapSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 1, 0, null, null, 900, 600); + false, 0, 1, 0, null, null, 900, 60, 300, 600, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index f744b05b72f1..61c2cf0635e2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -69,7 +69,7 @@ public record MetastoreSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 75, 30, null, null, 300, 60); + false, 1, 75, 30, null, null, 300, 60, 300, 60, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index 9827b7a7d733..716e4025c50d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 5, 10, null, null, 600, 120); + false, 0, 5, 10, null, null, 600, 60, 300, 120, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 2e641cdb4c1c..95e66bd91979 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -73,12 +73,27 @@ private ConfigUtils() { public static final String HIVE_LLAP_DAEMON_NUM_EXECUTORS_KEY = "hive.llap.daemon.num.executors"; + public static final String METASTORE_SERVER_TRANSPORT_MODE_KEY = "metastore.server.thrift.transport.mode"; + public static final String METASTORE_SERVER_TRANSPORT_MODE_DEFAULT = "http"; + + public static final String METASTORE_SERVER_HTTP_PATH_KEY = "metastore.server.thrift.http.path"; + public static final String METASTORE_SERVER_HTTP_PATH_DEFAULT = "metastore"; + + public static final String METASTORE_CLIENT_TRANSPORT_MODE_KEY = "hive.metastore.client.thrift.transport.mode"; + public static final String METASTORE_CLIENT_TRANSPORT_MODE_DEFAULT = "http"; + + public static final String METASTORE_CLIENT_HTTP_PATH_KEY = "metastore.client.thrift.http.path"; + public static final String METASTORE_CLIENT_HTTP_PATH_DEFAULT = "metastore"; + public static final String METASTORE_SERVER_MAX_THREADS_KEY = "metastore.server.max.threads"; public static final String METASTORE_SERVER_MAX_THREADS_HIVE_KEY = "hive.metastore.server.max.threads"; public static final int METASTORE_SERVER_MAX_THREADS_DEFAULT = 1000; public static final String HIVE_METASTORE_URIS_KEY = "hive.metastore.uris"; + public static final String HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY = "hive.server2.tez.sessions.per.default.queue"; + public static final int HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT = 1; + public static final String HIVE_SERVER2_THRIFT_PORT_KEY = "hive.server2.thrift.port"; public static final int HIVE_SERVER2_THRIFT_PORT_DEFAULT = 10000; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java index 7baaf2afc34f..f046b685f653 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/HiveConfigBuilder.java @@ -57,6 +57,11 @@ public static Map getHiveServer2HiveSite( if (metastoreUri != null && !metastoreUri.isEmpty()) { props.put(ConfigUtils.METASTORE_URIS_KEY, metastoreUri); } + // Client-side HTTP transport mode to match metastore server config. + props.put(ConfigUtils.METASTORE_CLIENT_TRANSPORT_MODE_KEY, + ConfigUtils.METASTORE_CLIENT_TRANSPORT_MODE_DEFAULT); + props.put(ConfigUtils.METASTORE_CLIENT_HTTP_PATH_KEY, + ConfigUtils.METASTORE_CLIENT_HTTP_PATH_DEFAULT); props.put(ConfigUtils.HIVE_METASTORE_WAREHOUSE_KEY, spec.metastore().warehouseDir()); props.put(ConfigUtils.HIVE_SERVER2_ENABLE_DOAS_KEY, "false"); @@ -163,6 +168,13 @@ public static Map getMetastoreSite(HiveClusterSpec spec) { MetastoreSpec metastore = spec.metastore(); Map props = new LinkedHashMap<>(); + // HTTP transport mode: stateless connections allow safe scale-down + // without breaking active client connections. + props.put(ConfigUtils.METASTORE_SERVER_TRANSPORT_MODE_KEY, + ConfigUtils.METASTORE_SERVER_TRANSPORT_MODE_DEFAULT); + props.put(ConfigUtils.METASTORE_SERVER_HTTP_PATH_KEY, + ConfigUtils.METASTORE_SERVER_HTTP_PATH_DEFAULT); + props.put(ConfigUtils.METASTORE_WAREHOUSE_KEY, metastore.warehouseDir()); From bde0ca347ffb86d5d9b873fb2a3653acc54c9cca Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 29 May 2026 22:23:07 +0530 Subject: [PATCH 06/22] Refactor --- .../dependent/HiveDependentResource.java | 27 ++-- .../operator/dependent/HivePdbDependent.java | 16 ++- .../HiveServer2HttpScaledObjectDependent.java | 130 ------------------ .../MetastoreDeploymentDependent.java | 2 +- 4 files changed, 32 insertions(+), 143 deletions(-) delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 3a47e4e114b4..8683c06d7bb2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -247,7 +247,7 @@ protected static String buildDrainScript( // the JVM — causing a full grace-period wait before SIGKILL. // Use 'java' pattern to avoid matching this script itself. lines.add("echo '[preStop] Sending SIGTERM to Java process...'"); - lines.add("kill $(pgrep -f 'java.*org.apache') 2>/dev/null"); + lines.add("pkill -f 'java.*org.apache' || true"); lines.add("exit 0"); return String.join("\n", lines); } @@ -311,7 +311,7 @@ protected static String buildDualMetricDrainScript( // the JVM — causing a full grace-period wait before SIGKILL. // Use 'java' pattern to avoid matching this script itself. lines.add("echo '[preStop] Sending SIGTERM to Java process...'"); - lines.add("kill $(pgrep -f 'java.*org.apache') 2>/dev/null"); + lines.add("pkill -f 'java.*org.apache' || true"); lines.add("exit 0"); return String.join("\n", lines); } @@ -803,12 +803,23 @@ private static String buildJmxExporterConfig(String component) { sb.append(" type: GAUGE\n"); break; case "llap": - // LLAP uses its own MetricsSystem (not DefaultMetricsSystem). - // Default JMX exporter pattern (.*) exports Hadoop Metrics2 MBeans as: - // hadoop_llapdaemon_{name=""} - // e.g., hadoop_llapdaemon_executornumqueuedrequests{name="LlapDaemonExecutorMetrics-..."} - // No custom rules needed — the default naming is usable directly. - sb.append("- pattern: '.*'\n"); + // Only export the executor metrics KEDA and the drain script need. + // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, + // causing CPU spikes and GC pressure on the LLAP JVM. + // Internal format: Hadoop><>Attribute + // Separate rules per attribute — JMX Exporter 1.x caches per-bean, not per-attribute. + sb.append("- pattern: 'Hadoop<>ExecutorNumQueuedRequests'\n"); + sb.append(" name: hadoop_llapdaemon_executornumqueuedrequests\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsConfigured'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsconfigured\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsAvailable'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsavailable\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutors'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); + sb.append(" type: GAUGE\n"); break; case "tezam": // TezAM DAG execution metrics diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java index 2942a5b674bf..e19413341c9b 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -74,7 +74,9 @@ protected PodDisruptionBudget desired(HiveCluster hiveCluster, + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class HiveServer2 extends HivePdbDependent { - public HiveServer2() { super("hiveserver2"); } + public HiveServer2() { + super("hiveserver2"); + } } @KubernetesDependent( @@ -82,7 +84,9 @@ public static class HiveServer2 extends HivePdbDependent { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class Metastore extends HivePdbDependent { - public Metastore() { super("metastore"); } + public Metastore() { + super("metastore"); + } } @KubernetesDependent( @@ -90,7 +94,9 @@ public static class Metastore extends HivePdbDependent { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class Llap extends HivePdbDependent { - public Llap() { super("llap"); } + public Llap() { + super("llap"); + } } @KubernetesDependent( @@ -98,6 +104,8 @@ public static class Llap extends HivePdbDependent { + "app.kubernetes.io/managed-by=hive-kubernetes-operator") ) public static class TezAm extends HivePdbDependent { - public TezAm() { super("tezam"); } + public TezAm() { + super("tezam"); + } } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java deleted file mode 100644 index 055bd878d2f3..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2HttpScaledObjectDependent.java +++ /dev/null @@ -1,130 +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.hive.kubernetes.operator.dependent; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA HTTPScaledObject for HiveServer2 scale-to-zero. - *

- * Requires the KEDA HTTP Add-on to be installed in the cluster. - * The HTTP Add-on creates an interceptor proxy that: - *

    - *
  • Sits in front of the HS2 Service
  • - *
  • Queues incoming beeline/HTTP requests when HS2 has 0 pods
  • - *
  • Triggers KEDA to scale HS2 from 0 to 1
  • - *
  • Forwards the queued request once a pod is ready
  • - *
- *

- * This dependent is activated ONLY when minReplicas == 0 (scale-to-zero mode). - * When minReplicas > 0, the regular ScaledObject (Prometheus-based) is used instead. - */ -public class HiveServer2HttpScaledObjectDependent extends HiveGenericDependentResource { - - public HiveServer2HttpScaledObjectDependent() { - super(new GroupVersionKind("http.keda.sh", "v1alpha1", "HTTPScaledObject")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); - int maxReplicas = hiveCluster.getSpec().hiveServer2().replicas(); - String clusterName = hiveCluster.getMetadata().getName(); - String namespace = hiveCluster.getMetadata().getNamespace(); - String deploymentName = clusterName + "-hiveserver2"; - String serviceName = clusterName + "-hiveserver2"; - - int httpPort = ConfigUtils.getInt( - hiveCluster.getSpec().hiveServer2().configOverrides(), - ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, - null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); - - Map spec = new HashMap<>(); - - // Hosts the interceptor matches for routing. - // Includes: internal service FQDN, short name, interceptor proxy name - // (for in-cluster kubectl exec), and localhost (for port-forward). - spec.put("hosts", List.of( - serviceName + "." + namespace + ".svc.cluster.local", - serviceName, - "keda-add-ons-http-interceptor-proxy.keda.svc", - "localhost" - )); - spec.put("pathPrefixes", List.of("/")); - - // Target deployment and service - spec.put("scaleTargetRef", Map.of( - "name", deploymentName, - "kind", "Deployment", - "apiVersion", "apps/v1", - "service", serviceName, - "port", httpPort - )); - - // Replica bounds - spec.put("replicas", Map.of( - "min", 0, - "max", maxReplicas - )); - - // Scaling metric: scale up when there are pending requests - spec.put("scalingMetric", Map.of( - "requestRate", Map.of( - "granularity", "1s", - "targetValue", autoscaling.scaleUpThreshold(), - "window", "1m" - ) - )); - - // Cooldown before scaling back to 0 - spec.put("scaledownPeriod", autoscaling.cooldownSeconds()); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("http.keda.sh/v1alpha1") - .withKind("HTTPScaledObject") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(namespace) - .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hiveserver2-httpso"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index 4b39d0341a51..34dcc099abcc 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -182,7 +182,7 @@ protected Deployment desired(HiveCluster hiveCluster, String preStopScript = String.join("\n", "#!/bin/bash", "echo '[preStop] Sending SIGTERM to Metastore Java process...'", - "kill $(pgrep -f 'java.*org.apache') 2>/dev/null", + "pkill -f 'java.*org.apache' || true", "exit 0"); applyAutoscalingLifecycle( deployment.getSpec().getTemplate().getSpec(), From cb9ff15fdcecc5487ac4f57b2b17291b099d1b46 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Sat, 30 May 2026 09:03:30 +0530 Subject: [PATCH 07/22] Remove Promethous and any other dependency --- packaging/src/kubernetes/README.md | 398 +++++------------- .../crds/hiveclusters.hive.apache.org-v1.yml | 256 ++++++----- .../hive-operator/templates/clusterrole.yaml | 12 +- .../hive-operator/templates/hivecluster.yaml | 31 +- .../kubernetes/helm/hive-operator/values.yaml | 56 +-- .../autoscaling/ComponentAutoscaler.java | 122 ++++++ .../autoscaling/HiveClusterAutoscaler.java | 207 +++++++++ .../HiveServer2ScalingStrategy.java | 63 +++ .../autoscaling/LlapScalingStrategy.java | 133 ++++++ .../autoscaling/MetastoreScalingStrategy.java | 101 +++++ .../operator/autoscaling/MetricsScraper.java | 142 +++++++ .../operator/autoscaling/PodMetrics.java | 25 ++ .../autoscaling/PrometheusTextParser.java | 128 ++++++ .../operator/autoscaling/ScalingStrategy.java | 55 +++ .../autoscaling/StabilizationWindow.java | 74 ++++ .../autoscaling/TezAmScalingStrategy.java | 122 ++++++ .../dependent/HiveDependentResource.java | 66 ++- .../HiveGenericDependentResource.java | 209 --------- .../dependent/HiveScaledObjectDependent.java | 373 ---------------- .../HiveServer2DeploymentDependent.java | 10 +- .../HiveServer2InterceptorRouteDependent.java | 116 ----- .../dependent/LlapStatefulSetDependent.java | 11 +- .../MetastoreDeploymentDependent.java | 9 +- .../dependent/TezAmStatefulSetDependent.java | 11 +- .../operator/model/spec/AutoscalingSpec.java | 36 +- .../operator/model/spec/HiveServer2Spec.java | 4 +- .../operator/model/spec/LlapSpec.java | 4 +- .../operator/model/spec/MetastoreSpec.java | 4 +- .../operator/model/spec/TezAmSpec.java | 4 +- .../model/status/AutoscalingStatus.java | 89 ++++ .../model/status/ComponentStatus.java | 54 ++- .../reconciler/HiveClusterReconciler.java | 189 +++++++-- .../operator/reconciler/HiveWorkflowSpec.java | 29 -- .../kubernetes/operator/util/ConfigUtils.java | 15 + 34 files changed, 1829 insertions(+), 1329 deletions(-) create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PodMetrics.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ScalingStrategy.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/StabilizationWindow.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java delete mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index f52da4db6eb6..54e58393fb85 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -510,90 +510,40 @@ kubectl describe hivecluster hive ## Autoscaling The operator supports metric-based autoscaling for all four Hive components using -[KEDA](https://keda.sh/) ScaledObjects and Kubernetes-native HPA. Autoscaling is +an **operator-driven control loop** that scrapes JMX Exporter metrics directly from +pods. No Prometheus server or external autoscaling tools are needed. Autoscaling is opt-in per component and designed for **zero query failures** during scale-down. ### Prerequisites -- [KEDA](https://keda.sh/) installed in the cluster -- [Prometheus](https://prometheus.io/) scraping Hive pod metrics (for HS2, HMS, LLAP custom metrics) -- Kubernetes metrics-server (for CPU-based triggers on Tez AM) -- [KEDA HTTP Add-on](https://github.com/kedacore/http-add-on) — **required for `minReplicas: 0`**, enables automatic wake-from-zero for HS2 +- No external dependencies — the operator handles all scaling decisions internally -### Installing KEDA +### How It Works -KEDA must be installed **before** enabling autoscaling on any Hive component. -The operator creates KEDA `ScaledObject` custom resources which require the KEDA -CRDs to be present on the cluster. - -```bash -# Add the KEDA Helm repo -helm repo add kedacore https://kedacore.github.io/charts -helm install keda kedacore/keda --namespace keda --create-namespace --wait -``` - -Verify KEDA is running: - -```bash -kubectl get pods -n keda -# Expected: keda-operator, keda-metrics-apiserver, keda-admission-webhooks -kubectl get crd | grep keda -# Expected: scaledobjects.keda.sh, scaledjobs.keda.sh, triggerauthentications.keda.sh, etc. -``` - -**For HS2 scale-to-zero** (`minReplicas: 0`), install the KEDA HTTP Add-on: - -```bash -helm install http-add-on kedacore/keda-add-ons-http \ - --namespace keda --wait -``` - -Verify the interceptor is running: - -```bash -kubectl get pods -n keda -l app=keda-add-ons-http-interceptor-proxy -# Expected: keda-add-ons-http-interceptor-proxy-... Running -``` - -> **Note:** The HTTP Add-on is required when `minReplicas: 0`. The operator creates -> an `InterceptorRoute` CRD that configures the interceptor proxy to route traffic -> to HS2. When HS2 has zero pods, the interceptor holds incoming requests and triggers -> scale-up via an `external-push` trigger on the HS2 ScaledObject. The first request -> takes ~30-60s while the pod starts. - -**For Prometheus-based triggers** (HS2, HMS, LLAP), install Prometheus: - -```bash -helm repo add prometheus-community https://prometheus-community.github.io/helm-charts -helm install prometheus prometheus-community/prometheus \ - --namespace monitoring --create-namespace --wait -``` - -> **Note:** If autoscaling is enabled in the HiveCluster spec but KEDA is not -> installed, the operator will fail to reconcile with errors like -> `"Could not find the metadata for the given apiVersion and kind"`. -> Always install KEDA before setting `autoscaling.enabled: true`. +When `autoscaling.enabled: true` is set for a component, the operator: +1. Attaches the JMX Exporter javaagent (port 9404) to each pod +2. Polls `/metrics` on each pod at `metricsScrapeIntervalSeconds` intervals +3. Computes desired replicas using component-specific formulas +4. Applies HPA-like stabilization windows (scale-up/scale-down) +5. Patches the workload `spec.replicas` directly ### Graceful Scale-Down Architecture ``` -┌─────────────────────────────────────────────────────────────────────┐ -│ Scale Down Flow │ -├─────────────────────────────────────────────────────────────────────┤ -│ 1. KEDA reduces desired replicas (cooldown elapsed, metric below │ -│ threshold) │ -│ 2. PodDisruptionBudget ensures minAvailable=1 (at least one pod │ -│ always running) │ -│ 3. Kubernetes sends SIGTERM to selected pod │ -│ 4. preStop hook runs: │ -│ - HS2: deregisters from ZK, drains open sessions, kills JVM │ -│ - HMS: kills JVM (stateless HTTP — no drain needed) │ -│ - LLAP: waits until all executors become idle, kills JVM │ -│ - TezAM: waits for current DAG completion, kills JVM │ -│ 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety cap) │ -│ 6. Pod terminates immediately once drain completes (does NOT wait │ -│ the full grace period — it's only the upper safety bound) │ -└─────────────────────────────────────────────────────────────────────┘ + Scale Down Flow + 1. Operator reduces desired replicas (metric below threshold, + stabilization window elapsed) + 2. PodDisruptionBudget ensures minAvailable=1 (at least one pod + always running) + 3. Kubernetes sends SIGTERM to selected pod + 4. preStop hook runs: + - HS2: deregisters from ZK, drains open sessions, kills JVM + - HMS: kills JVM (stateless HTTP — no drain needed) + - LLAP: waits until all executors become idle, kills JVM + - TezAM: waits for current DAG completion, kills JVM + 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety cap) + 6. Pod terminates immediately once drain completes (does NOT wait + the full grace period — it's only the upper safety bound) ``` > **Note:** Shell entrypoints (PID 1) in containers don't forward SIGTERM to child @@ -603,113 +553,89 @@ helm install prometheus prometheus-community/prometheus \ ### Scaling Timers -The autoscaling system uses four independent timing controls: +The autoscaling system uses three independent timing controls: | Timer | Config Field | Default | Purpose | |-------|-------------|---------|---------| -| **Metrics scrape interval** | `metricsScrapeIntervalSeconds` | `10` | How often Prometheus scrapes the pod's metrics endpoint. This is the **biggest bottleneck** for autoscaling reaction time — KEDA cannot detect metric changes faster than the scrape interval. | -| **Scale-up stabilization** | `scaleUpStabilizationSeconds` | `60` | HPA window: picks the highest recommendation within this period before scaling up. Prevents flapping when metrics oscillate. Set to `0` for LLAP and TezAM (reactive dependents). | -| **Scale-down stabilization** | `scaleDownStabilizationSeconds` | `300` | HPA window: picks the highest recommendation within this period before scaling down. Prevents premature removal of pods during temporary load dips. | -| **KEDA cooldown** | `cooldownSeconds` | `300-900` | Time after **all** KEDA triggers go inactive (metric = 0) before KEDA scales from 1→0. Only relevant when `minReplicas: 0`. HPA handles N→1 transitions using the stabilization window. | +| **Metrics scrape interval** | `metricsScrapeIntervalSeconds` | `10` | How often the operator scrapes JMX Exporter `/metrics` on each pod. This is the **biggest bottleneck** for autoscaling reaction time. | +| **Scale-up stabilization** | `scaleUpStabilizationSeconds` | `60` | Window: picks the highest recommendation within this period before scaling up. Prevents flapping when metrics oscillate. Set to `0` for LLAP and TezAM (reactive dependents). | +| **Scale-down stabilization** | `scaleDownStabilizationSeconds` | `300-900` | Window: picks the most conservative (highest) recommendation within this period before scaling down. Also acts as the cooldown between consecutive scale-downs — no separate cooldown needed. | **How they interact:** -- Load spike detected → Prometheus scrapes metric within `metricsScrapeIntervalSeconds` → HPA waits `scaleUpStabilizationSeconds` then scales up -- Load drops → HPA waits `scaleDownStabilizationSeconds` then scales down (N→1) -- All triggers inactive → KEDA waits `cooldownSeconds` then scales 1→0 +- Load spike detected → operator scrapes metrics within `metricsScrapeIntervalSeconds` → waits `scaleUpStabilizationSeconds` then scales up +- Load drops → operator waits `scaleDownStabilizationSeconds` (stabilization window must confirm low demand consistently) then scales down -**Tuning reaction time:** With defaults (`metricsScrapeIntervalSeconds: 10`, `scaleUpStabilizationSeconds: 0` for LLAP/TezAM), scale-up latency is ~15s (one scrape + KEDA polling). For HS2 with `scaleUpStabilizationSeconds: 60`, expect ~70s. Reducing `metricsScrapeIntervalSeconds` below 10 gives diminishing returns and increases Prometheus load. +**Tuning reaction time:** With defaults (`metricsScrapeIntervalSeconds: 10`, `scaleUpStabilizationSeconds: 0` for LLAP/TezAM), scale-up latency is ~10-20s (one scrape cycle). For HS2 with `scaleUpStabilizationSeconds: 60`, expect ~70s. ### Per-Component Scaling Logic -| Component | Scale-Up Trigger | Scale-Down Trigger | Native Metric | -|-----------|-----------------|-------------------|---------------| -| **HiveServer2** | `sum(hs2_open_sessions)` > scaleUpThreshold **OR** CPU > targetCpuValue | Sessions below threshold **AND** CPU below activationCpuValue | `hs2_open_sessions` (sum across pods) | -| **Metastore** | API request rate > scaleUpThreshold **OR** CPU > targetCpuValue | Request rate below threshold **AND** CPU below activationCpuValue | `api_*_total` (manual delta for Prometheus 3.x compatibility) | -| **LLAP** | Total busy slots > scaleUpThreshold (queued + busy executors) | All executors idle + no HS2 sessions | `NumQueuedRequests`, `NumExecutorsConfigured`, `NumExecutorsAvailable` | -| **Tez AM** | max(`sum(hs2_open_sessions)`, `count(HS2 pods)` x `sessions.per.default.queue`) | All HS2 sessions closed | `hs2_open_sessions` (demand-driven, no CPU trigger) | +| Component | Scale-Up Formula | Scale-Down | JMX Metric | +|-----------|-----------------|------------|------------| +| **HiveServer2** | `ceil(sum(hs2_open_sessions) / scaleUpThreshold)` | Sessions drop to 0 → scale to minReplicas | `hs2_open_sessions` | +| **Metastore** | `ceil(api_request_rate / scaleUpThreshold)` | Rate drops to 0 → scale to minReplicas | `api_*_total` (operator computes delta/time) | +| **LLAP** | `ceil(avg(queued + configured - available) / scaleUpThreshold)` | All executors idle + no HS2 sessions | `hadoop_llapdaemon_executor*` | +| **Tez AM** | `max(sum(hs2_open_sessions), count(HS2_pods) * sessions_per_queue)` | All HS2 sessions closed | `hs2_open_sessions` (from HS2 pods) | -**TezAM Scaling Model:** TezAM uses demand-driven scaling with two KEDA triggers (max wins): +**TezAM Scaling Model:** TezAM uses demand-driven scaling with two formulas (max wins): 1. **Session demand** — `sum(hs2_open_sessions)`: scales to match the total number of concurrent sessions across all HS2 pods (each session needs its own exclusive TezAM). -2. **Pre-warm** — `count(HS2 pods) × hive.server2.tez.sessions.per.default.queue` (default 1): - ensures every HS2 pod has enough TezAM sessions pre-claimed from ZooKeeper before queries arrive. +2. **Pre-warm** — `count(HS2 pods with sessions) × hive.server2.tez.sessions.per.default.queue` (default 1): + ensures every active HS2 pod has enough TezAM sessions pre-claimed from ZooKeeper. -KEDA takes the maximum desired replicas across both triggers. This ensures TezAM capacity -is always sufficient for both current demand and eager session pre-warming. No CPU-based -trigger is used — TezAM scaling is purely demand-driven from HS2 metrics. +The operator takes the maximum across both formulas. This ensures TezAM capacity +is always sufficient for both current demand and eager session pre-warming. +TezAM scaling is purely demand-driven from HS2 metrics. ### Scale-to-Zero Architecture -When `minReplicas: 0` is configured (default for HS2, LLAP, TezAM), the cluster -scales down to zero pods when completely idle. The operator uses a **unified -ScaledObject + InterceptorRoute** architecture — a single KEDA ScaledObject per -component handles both Prometheus-based scaling and wake-from-zero, while an -`InterceptorRoute` (from the KEDA HTTP Add-on) provides routing-only configuration -without creating a conflicting second ScaledObject. +When `minReplicas: 0` is configured (LLAP, TezAM), the cluster scales those +components down to zero pods when HS2 has no active sessions. HS2 itself always +maintains at least 1 replica (`minReplicas >= 1`) so it is always available to +accept connections. ``` Scale-to-Zero (Idle Detection) - 1. No open sessions/queries for cooldownPeriod seconds - → KEDA detects all triggers inactive - → scales HS2 to 0 (idleReplicaCount) + 1. HS2 reports hs2_open_sessions = 0 for scaleDownStabilization + → operator scales HS2 to minReplicas (>= 1) - 2. LLAP/TezAM ScaledObjects see hs2_open_sessions = 0 - → activation triggers inactive for cooldownPeriod - → scale LLAP and TezAM to 0 + 2. Operator sees hs2_open_sessions = 0 on next LLAP/TezAM eval + → activation gate fails + → scale LLAP and TezAM to 0 (if minReplicas=0) 3. HMS stays at minReplicas=1 (always available) ``` ``` - Wake-from-Zero (with KEDA HTTP Add-on) + Wake-from-Zero (LLAP/TezAM) - 1. Beeline connects → KEDA HTTP interceptor proxy queues the - request and triggers HS2 scale-up via external-push trigger + 1. Beeline connects to HS2 (always running, at least 1 pod) - 2. HS2 pod starts, reports hs2_open_sessions > 0 to Prometheus + 2. HS2 reports hs2_open_sessions > 0 via JMX Exporter - 3. KEDA detects cross-component activation trigger: - - LLAP ScaledObject sees hs2_open_sessions > 0 → scales up - - TezAM ScaledObject sees hs2_open_sessions > 0 → scales up + 3. Operator detects HS2 sessions on next scrape cycle: + - LLAP activation gate passes → scales up from 0 + - TezAM activation gate passes → scales up from 0 4. Query executes once LLAP/TezAM pods are ready ``` -The HS2 ScaledObject combines three trigger types in a single resource: -- **Prometheus trigger** (`sum(hs2_open_sessions)`) — session-aware scaling using total - session count across all pods (`sum()` prevents premature scale-down of pods with - active sessions; `desired = ceil(sum / threshold)`) -- **CPU trigger** (`AverageValue` in millicores) — load-based scaling when `targetCpuValue` is configured -- **external-push trigger** — wake-from-zero via the KEDA HTTP Add-on interceptor - **Session protection:** The HS2 Service uses `sessionAffinity: ClientIP` to ensure beeline clients always reach the same pod. The preStop hook deregisters the pod from ZooKeeper (preventing new sessions) and waits for `hs2_open_sessions` to drain to 0 before terminating. The `gracePeriodSeconds` (default 3600s) is a safety cap — the pod terminates immediately once sessions drain, not after the full grace period. -The `InterceptorRoute` CRD (`http.keda.sh/v1beta1`) configures only the interceptor -routing (host matching, backend target) without auto-creating a ScaledObject — this -avoids the dual-HPA conflict that `HTTPScaledObject` would cause. - -> **Important:** Automatic wake-from-zero requires the KEDA HTTP Add-on. Traffic -> must flow through the interceptor proxy (via Ingress or port-forward). Without the -> HTTP Add-on, HS2 must be manually woken (`kubectl scale deployment/hive-hiveserver2 --replicas=1`). -> LLAP and TezAM wake automatically once HS2 reports open sessions. See -> [Connect to HiveServer2 > Connecting with Scale-to-Zero](#connecting-with-scale-to-zero-minreplicas--0) -> for setup instructions. - **Component-specific behavior:** | Component | minReplicas | Scale-to-Zero Trigger | Wake Trigger | |-----------|-------------|----------------------|--------------| -| **HS2** | 0 | `hs2_open_sessions = 0` for cooldown | HTTP request via KEDA interceptor (`external-push`) | +| **HS2** | 1 | N/A (always running) | N/A | | **HMS** | 1 | Never (always running) | N/A | -| **LLAP** | 0 | `hs2_open_sessions = 0` for cooldown | `hs2_open_sessions > 0` (cross-component) | -| **TezAM** | 0 | No HS2 pods with open sessions | `hs2_open_sessions > 0` (cross-component, demand-driven) | +| **LLAP** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | +| **TezAM** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | ### Enabling Autoscaling @@ -778,24 +704,22 @@ cluster: replicas: 10 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - # minReplicas: 0 # default — scale to zero when idle (requires KEDA HTTP Add-on) + # minReplicas: 1 # default — always keep at least 1 HS2 running # scaleUpThreshold: 80 # default — avg open sessions per pod triggering scale-up - # cooldownSeconds: 600 # default — KEDA 1→0 cooldown after all triggers inactive - # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window - # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window - # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) + # scaleUpStabilizationSeconds: 60 # default — scale-up window + # scaleDownStabilizationSeconds: 600 # default — scale-down window (also acts as cooldown) + # metricsScrapeIntervalSeconds: 10 # default — operator scrape interval (lower = faster reaction) metastore: replicas: 6 # Acts as maxReplicas when autoscaling is enabled autoscaling: enabled: true - # minReplicas: 0 # default — scale to zero when no connections + # minReplicas: 1 # default — always keep at least 1 metastore running # scaleUpThreshold: 75 # default — API request rate (req/s) triggering scale-up - # cooldownSeconds: 300 # default — KEDA 1→0 cooldown after all triggers inactive - # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window - # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window + # scaleUpStabilizationSeconds: 60 # default — scale-up window + # scaleDownStabilizationSeconds: 300 # default — scale-down window (also acts as cooldown) # gracePeriodSeconds: 60 # default — fast drain (HMS is stateless) - # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval + # metricsScrapeIntervalSeconds: 10 # default — operator scrape interval llap: replicas: 8 # Acts as maxReplicas when autoscaling is enabled @@ -803,11 +727,10 @@ cluster: enabled: true # minReplicas: 0 # default — scale to zero when no HS2 sessions # scaleUpThreshold: 1 # default — total busy slots (queued+running) triggering scale-up - # cooldownSeconds: 900 # default — KEDA 1→0 cooldown (scaling down destroys cache) - # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window - # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window + # scaleUpStabilizationSeconds: 60 # default — scale-up window + # scaleDownStabilizationSeconds: 900 # default — scale-down window (long — scaling down destroys cache) # gracePeriodSeconds: 600 # default — 10 min drain for in-flight fragments - # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) + # metricsScrapeIntervalSeconds: 10 # default — operator scrape interval (lower = faster reaction) tezAm: replicas: 10 # Acts as maxReplicas when autoscaling is enabled @@ -818,7 +741,7 @@ cluster: # scaleUpStabilizationSeconds: 60 # default — HPA scale-up window # scaleDownStabilizationSeconds: 300 # default — HPA scale-down window # gracePeriodSeconds: 120 # default — 2 min drain for DAG completion - # metricsScrapeIntervalSeconds: 10 # default — Prometheus scrape interval (lower = faster reaction) + # metricsScrapeIntervalSeconds: 10 # default — operator scrape interval (lower = faster reaction) ``` ```bash @@ -826,85 +749,45 @@ helm install hive ./helm/hive-operator -f values-autoscaling.yaml ``` When autoscaling is enabled, the operator automatically: -- Deploys the Prometheus JMX Exporter agent sidecar (port 9404, `/metrics`) +- Deploys the JMX Exporter javaagent (port 9404, `/metrics`) - Enables `hive.server2.metrics.enabled` / `metastore.metrics.enabled` (JMX reporter) -- Adds Prometheus scrape annotations to pods (including `prometheus.io/scrape-interval` for fast reaction) -- Creates KEDA ScaledObjects with the configured thresholds +- Attaches JMX Exporter javaagent (port 9404, `/metrics`) to each pod - Creates PodDisruptionBudgets (minAvailable: 1) - Configures preStop lifecycle hooks for graceful drain - Sets `terminationGracePeriodSeconds` to the configured grace period -- Adds cross-component activation triggers for LLAP/TezAM (wake when HS2 has open sessions) +- LLAP/TezAM use HS2 metrics as activation gate (only scale when HS2 has sessions) -**Exported Prometheus Metrics (per component):** +**JMX Metrics Scraped by Operator (per component):** | Component | Key Metrics | Purpose | |-----------|---------|---------| -| **HiveServer2** | `hs2_open_sessions` | Session count — used by HS2 ScaledObject (sum for scale-up protection) and TezAM ScaledObject (demand-driven scaling) | -| **Metastore** | `api_*_total` | API call counters (manual delta for Prometheus 3.x compatibility) | -| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available (scaling trigger) | +| **HiveServer2** | `hs2_open_sessions` | Session count — used for HS2 scaling and as activation gate for LLAP/TezAM | +| **Metastore** | `api_*_total` | API call counters (operator computes request rate from deltas) | +| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available | | **Tez AM** | N/A (scales on HS2 metrics) | TezAM scaling is demand-driven from `hs2_open_sessions` — no TezAM-specific metrics needed | -### CPU-Based Scaling - -The operator can include a **CPU trigger** in the ScaledObject for HS2 and Metastore. -The trigger uses KEDA's `AverageValue` metric type with **absolute millicore targets** that -you specify directly. This handles burstable QoS pods correctly — unlike `Utilization` -(which measures against the CPU request), `AverageValue` uses actual CPU consumption in -absolute terms, so pods with a small request but high limit won't show perpetual >100% -utilization that prevents scale-down. - -**The CPU trigger is opt-in:** it is only added to the ScaledObject when you explicitly set -both `targetCpuValue` and `activationCpuValue` in the autoscaling config. If omitted, the -operator relies solely on the Prometheus-based trigger (sessions, connections, etc.). +### Enabling Autoscaling — Example -**How it works:** - -- `targetCpuValue` — the average CPU per pod (e.g., `"1500m"` or `"1"`) that triggers scale-up -- `activationCpuValue` — below this CPU value, the trigger is completely inactive - (doesn't participate in scaling decisions at all) -- Both the CPU trigger and the Prometheus-based trigger are evaluated independently — - if **either** exceeds its threshold, the component scales up (OR logic) -- Scale-down only happens when **both** triggers agree load is low -- The component must also have `resources` defined on its pods; if `targetCpuValue` is set - but `resources` is missing, the operator logs a warning and skips the CPU trigger - -**Example:** With `targetCpuValue: "1600m"` and `activationCpuValue: "400m"`, KEDA scales up -when average pod CPU exceeds 1600m and considers the trigger inactive below 400m. - -To enable both Prometheus and CPU-based scaling: +To enable autoscaling for HS2 and Metastore: ```yaml cluster: hiveServer2: - resources: - requestsCpu: "500m" - limitsCpu: "2" - requestsMemory: "2Gi" + replicas: 4 # max replicas ceiling autoscaling: enabled: true scaleUpThreshold: 1 # scale up when total sessions > 1 - targetCpuValue: "1600m" # scale up when avg CPU > 1600m per pod - activationCpuValue: "400m" # CPU trigger inactive below 400m + minReplicas: 1 # always keep at least 1 HS2 pod running metastore: - resources: - requestsCpu: "500m" - limitsCpu: "1" - requestsMemory: "1Gi" + replicas: 3 # max replicas ceiling autoscaling: enabled: true - targetCpuValue: "750m" - activationCpuValue: "200m" + minReplicas: 1 # always keep at least 1 running + scaleUpThreshold: 75 # API requests/sec threshold ``` -| Setting | Effect on CPU trigger | -|---------|----------------------| -| `targetCpuValue` | Absolute CPU target (e.g., `"1500m"` or `"1"`). **Required** to enable CPU trigger. | -| `activationCpuValue` | CPU below which trigger is inactive. **Required** with targetCpuValue. | -| `resources` | Pod resources must be defined — operator warns and skips CPU trigger otherwise. | - -> **Note:** LLAP and TezAM scaling use only Prometheus-based triggers and do not -> include CPU triggers. LLAP scales on total busy slots (queued + running executors). +> **Note:** LLAP scales on total busy slots (queued + running executors). > TezAM scales on demand — the number of active HS2 pods multiplied by > `hive.server2.tez.sessions.per.default.queue` (default 1). @@ -913,14 +796,13 @@ cluster: | Value | Default | Description | |-------|---------|-------------| | `cluster..replicas` | `1-2` | Static replica count, or max replicas ceiling when autoscaling is enabled | -| `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling | -| `cluster..autoscaling.minReplicas` | `0` (HS2/LLAP/TezAM), `1` (HMS) | Minimum replica count. Set to 0 for scale-to-zero | +| `cluster..autoscaling.enabled` | `false` | Enable operator-driven autoscaling | +| `cluster..autoscaling.minReplicas` | `1` (HS2/HMS), `0` (LLAP/TezAM) | Minimum replica count. Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum is 1) | | `cluster..autoscaling.scaleUpThreshold` | varies | Metric threshold triggering scale-up | -| `cluster..autoscaling.scaleDownThreshold` | varies | Metric threshold triggering scale-down | -| `cluster..autoscaling.cooldownSeconds` | varies | KEDA cooldown: seconds after all triggers go inactive before scaling 1→0 | -| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | HPA stabilization window for scale-up (picks highest recommendation in window) | -| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300` | HPA stabilization window for scale-down (picks highest recommendation in window) | -| `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap: max drain time before forced termination. Pod exits immediately once sessions/connections drain to 0. | +| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | Stabilization window for scale-up (picks highest recommendation in window) | +| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down (picks most conservative recommendation in window). Also acts as cooldown between consecutive scale-downs. | +| `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap: max drain time before forced termination. Pod exits immediately once drain completes. | +| `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | How often the operator scrapes JMX metrics from pods. Lower = faster reaction. | --- @@ -944,73 +826,14 @@ kubectl port-forward svc/hive-hiveserver2 10001:10001 beeline -u "jdbc:hive2://localhost:10001/;transportMode=http;httpPath=cliservice" ``` -### Connecting with Scale-to-Zero (minReplicas = 0) - -When HS2 is configured with `minReplicas: 0`, the deployment starts with zero pods. -Connections go through the **KEDA HTTP interceptor proxy** which automatically wakes -HS2 when a request arrives (first request takes ~30-60s while the pod starts). - -``` -Traffic flow: -Client → KEDA HTTP Interceptor → (if 0 pods: scale up, wait) → HS2 Service → HS2 Pod -``` - -**Via kubectl exec (no local Hive install needed):** - -The Metastore pod is always running (`minReplicas=1`) and has beeline pre-installed. -Connecting through the interceptor wakes HS2 from zero automatically: - -```bash -kubectl exec -it deploy/hive-metastore -- beeline -u "jdbc:hive2://keda-add-ons-http-interceptor-proxy.keda.svc:8080/;transportMode=http;httpPath=cliservice" -``` - -Or connect directly when HS2 is already running: - -```bash -kubectl exec -it deploy/hive-metastore -- beeline -u "jdbc:hive2://hive-hiveserver2:10001/;transportMode=http;httpPath=cliservice" -``` - -**Via port-forward (local development):** +### LLAP/TezAM Scale-to-Zero Behavior -```bash -# Port-forward the KEDA HTTP interceptor proxy -kubectl port-forward -n keda svc/keda-add-ons-http-interceptor-proxy 8080:8080 - -# Connect — interceptor auto-wakes HS2 (first request may take 30-60s) -beeline -u "jdbc:hive2://localhost:8080/;transportMode=http;httpPath=cliservice" -``` - -**Via Ingress:** - -Create an Ingress that routes to the KEDA interceptor. Uses [nip.io](https://nip.io) -wildcard DNS so no `/etc/hosts` editing is needed — `hive.127.0.0.1.nip.io` resolves -to `127.0.0.1` automatically: - -```bash -kubectl create ingress hive-interceptor -n keda --class=nginx \ - --rule="hive.127.0.0.1.nip.io/*=keda-add-ons-http-interceptor-proxy:8080" \ - --annotation="nginx.ingress.kubernetes.io/upstream-vhost=hive-hiveserver2.default.svc.cluster.local" -``` - -> The `upstream-vhost` annotation rewrites the Host header to the internal service -> name so the KEDA interceptor can match and route the request. +When LLAP and TezAM are configured with `minReplicas: 0` (the default), they start +with zero pods on fresh install. The operator automatically scales them up when HS2 +reports open sessions, and scales them back to zero when HS2 is idle. -Connect via beeline using the Ingress: - -```bash -beeline -u "jdbc:hive2://hive.127.0.0.1.nip.io:80/;transportMode=http;httpPath=cliservice" -``` - -> For production, replace `hive.127.0.0.1.nip.io` with your actual domain -> (e.g., `hive.example.com`) and ensure DNS points to your ingress controller. - -**Manual wake (fallback without HTTP Add-on):** - -```bash -kubectl scale deployment/hive-hiveserver2 --replicas=1 -kubectl wait --for=condition=ready pod -l app.kubernetes.io/component=hiveserver2 --timeout=120s -kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hiveserver2:10001/;transportMode=http;httpPath=cliservice" -``` +Since HS2 always runs at least 1 pod (`minReplicas >= 1`), no special connection +setup is needed — simply connect to HS2 and the operator wakes LLAP/TezAM as needed. > **Note:** The operator sets `hive.server2.transport.mode=http`, > `hive.server2.thrift.http.port=10001`, and @@ -1132,17 +955,13 @@ kubectl exec -it deployment/hive-hiveserver2 -- beeline -u "jdbc:hive2://hive-hi | Value | Default | Description | |-------|---------|-------------| -| `cluster..autoscaling.enabled` | `false` | Enable KEDA-based autoscaling for this component | -| `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (HS2 requires KEDA HTTP Add-on) | +| `cluster..autoscaling.enabled` | `false` | Enable operator-driven autoscaling for this component | +| `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (LLAP, TezAM only; HS2 minimum is 1) | | `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (total sessions for HS2, request rate for HMS, busy slots for LLAP, demand per HS2 pod for TezAM) | -| `cluster..autoscaling.scaleDownThreshold` | `30` | Prometheus metric threshold for scale-down (component-specific) | -| `cluster..autoscaling.targetCpuValue` | — | Absolute CPU target for scale-up (e.g., `1500m`). Omit to disable CPU trigger. | -| `cluster..autoscaling.activationCpuValue` | — | CPU value below which CPU trigger is inactive. Required with targetCpuValue. | -| `cluster..autoscaling.cooldownSeconds` | `300-900` | KEDA cooldown: seconds after all triggers go inactive before scaling 1→0 | -| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | HPA stabilization window for scale-up decisions (prevents flapping) | -| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300` | HPA stabilization window for scale-down decisions (prevents premature scale-down) | +| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | Stabilization window for scale-up decisions (prevents flapping) | +| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down decisions (also acts as cooldown between consecutive scale-downs) | | `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap (seconds) — pod terminates immediately once drain completes, this is only the upper bound | -| `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | Prometheus scrape interval override for this component's pods. Lower values make autoscaling react faster but increase Prometheus load. Applied via `prometheus.io/scrape-interval` pod annotation. | +| `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | How often the operator polls JMX metrics from pods. Lower = faster reaction time. | --- @@ -1184,20 +1003,13 @@ helm install hive ./helm/hive-operator -f my-values.yaml ```bash kubectl delete hivecluster --all -A --wait=false --ignore-not-found -kubectl delete ingress hive-interceptor -n keda --ignore-not-found helm uninstall hive --ignore-not-found kubectl delete crd hiveclusters.hive.apache.org --wait=false --ignore-not-found -kubectl delete crd --wait=false --ignore-not-found scaledobjects.keda.sh scaledjobs.keda.sh triggerauthentications.keda.sh clustertriggerauthentications.keda.sh httpscaledobjects.http.keda.sh interceptorroutes.http.keda.sh -helm uninstall http-add-on -n keda --ignore-not-found -helm uninstall keda -n keda --ignore-not-found -helm uninstall prometheus -n monitoring --ignore-not-found helm uninstall ozone --ignore-not-found helm uninstall postgres --ignore-not-found helm uninstall zookeeper --ignore-not-found kubectl delete pvc data-zookeeper-0 data-postgres-postgresql-0 --ignore-not-found kubectl delete secret hive-db-secret --ignore-not-found -kubectl delete namespace keda --wait=false --ignore-not-found -kubectl delete namespace monitoring --wait=false --ignore-not-found ``` --- diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 1e496830f21b..1faee974870c 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -45,18 +45,9 @@ spec: description: HiveServer2 component configuration properties: autoscaling: - description: Autoscaling configuration (requires KEDA installed - in the cluster) + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" properties: - activationCpuValue: - description: CPU average value below which the trigger is - inactive. Required if targetCpuValue is set. - type: string - cooldownSeconds: - default: 600 - description: Cooldown period in seconds after all KEDA triggers - are inactive before scaling from 1 to 0 (scale-to-zero delay) - type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -70,43 +61,35 @@ spec: type: integer metricsScrapeIntervalSeconds: default: 10 - description: Prometheus scrape interval in seconds for this - component's metrics. Lower values make autoscaling react - faster but increase Prometheus load. + description: How often (seconds) the operator scrapes JMX + metrics from pods. Lower values make autoscaling react faster. type: integer minReplicas: default: 0 - description: Minimum number of replicas (floor for scale-down). - Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on - for wake-from-zero) + description: "Minimum number of replicas (floor for scale-down).\ + \ Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum\ + \ is 1)" type: integer scaleDownStabilizationSeconds: - default: 300 + default: 600 description: Stabilization window in seconds for scale-down - decisions. HPA picks the highest recommendation within this - window to prevent premature scale-down. - type: integer - scaleDownThreshold: - default: 20 - description: Threshold that triggers scale-down for Prometheus-based - metrics + decisions. How long metrics must consistently indicate fewer + replicas before scale-down occurs. Also acts as the cooldown + between consecutive scale-downs. type: integer scaleUpStabilizationSeconds: default: 60 description: Stabilization window in seconds for scale-up - decisions. HPA picks the highest recommendation within this + decisions. Picks the highest recommendation within this window to prevent flapping. type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, connections for HMS, queue depth for\ - \ LLAP, pending tasks for TezAM)" + \ sessions per pod for HS2, request rate for HMS, busy slots\ + \ per daemon for LLAP). Not used by TezAM (demand-based:\ + \ 1 TezAM per session)." type: integer - targetCpuValue: - description: "Target CPU average value for scaling (e.g.,\ - \ '1500m' or '1'). If omitted, CPU scaling is disabled." - type: string type: object configOverrides: additionalProperties: @@ -217,18 +200,9 @@ spec: description: LLAP daemon configuration. Enabled by default. properties: autoscaling: - description: Autoscaling configuration (requires KEDA installed - in the cluster) + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" properties: - activationCpuValue: - description: CPU average value below which the trigger is - inactive. Required if targetCpuValue is set. - type: string - cooldownSeconds: - default: 600 - description: Cooldown period in seconds after all KEDA triggers - are inactive before scaling from 1 to 0 (scale-to-zero delay) - type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -242,43 +216,35 @@ spec: type: integer metricsScrapeIntervalSeconds: default: 10 - description: Prometheus scrape interval in seconds for this - component's metrics. Lower values make autoscaling react - faster but increase Prometheus load. + description: How often (seconds) the operator scrapes JMX + metrics from pods. Lower values make autoscaling react faster. type: integer minReplicas: default: 0 - description: Minimum number of replicas (floor for scale-down). - Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on - for wake-from-zero) + description: "Minimum number of replicas (floor for scale-down).\ + \ Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum\ + \ is 1)" type: integer scaleDownStabilizationSeconds: - default: 300 + default: 600 description: Stabilization window in seconds for scale-down - decisions. HPA picks the highest recommendation within this - window to prevent premature scale-down. - type: integer - scaleDownThreshold: - default: 20 - description: Threshold that triggers scale-down for Prometheus-based - metrics + decisions. How long metrics must consistently indicate fewer + replicas before scale-down occurs. Also acts as the cooldown + between consecutive scale-downs. type: integer scaleUpStabilizationSeconds: default: 60 description: Stabilization window in seconds for scale-up - decisions. HPA picks the highest recommendation within this + decisions. Picks the highest recommendation within this window to prevent flapping. type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, connections for HMS, queue depth for\ - \ LLAP, pending tasks for TezAM)" + \ sessions per pod for HS2, request rate for HMS, busy slots\ + \ per daemon for LLAP). Not used by TezAM (demand-based:\ + \ 1 TezAM per session)." type: integer - targetCpuValue: - description: "Target CPU average value for scaling (e.g.,\ - \ '1500m' or '1'). If omitted, CPU scaling is disabled." - type: string type: object configOverrides: additionalProperties: @@ -364,18 +330,9 @@ spec: description: Metastore component configuration properties: autoscaling: - description: Autoscaling configuration (requires KEDA installed - in the cluster) + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" properties: - activationCpuValue: - description: CPU average value below which the trigger is - inactive. Required if targetCpuValue is set. - type: string - cooldownSeconds: - default: 600 - description: Cooldown period in seconds after all KEDA triggers - are inactive before scaling from 1 to 0 (scale-to-zero delay) - type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -389,43 +346,35 @@ spec: type: integer metricsScrapeIntervalSeconds: default: 10 - description: Prometheus scrape interval in seconds for this - component's metrics. Lower values make autoscaling react - faster but increase Prometheus load. + description: How often (seconds) the operator scrapes JMX + metrics from pods. Lower values make autoscaling react faster. type: integer minReplicas: default: 0 - description: Minimum number of replicas (floor for scale-down). - Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on - for wake-from-zero) + description: "Minimum number of replicas (floor for scale-down).\ + \ Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum\ + \ is 1)" type: integer scaleDownStabilizationSeconds: - default: 300 + default: 600 description: Stabilization window in seconds for scale-down - decisions. HPA picks the highest recommendation within this - window to prevent premature scale-down. - type: integer - scaleDownThreshold: - default: 20 - description: Threshold that triggers scale-down for Prometheus-based - metrics + decisions. How long metrics must consistently indicate fewer + replicas before scale-down occurs. Also acts as the cooldown + between consecutive scale-downs. type: integer scaleUpStabilizationSeconds: default: 60 description: Stabilization window in seconds for scale-up - decisions. HPA picks the highest recommendation within this + decisions. Picks the highest recommendation within this window to prevent flapping. type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, connections for HMS, queue depth for\ - \ LLAP, pending tasks for TezAM)" + \ sessions per pod for HS2, request rate for HMS, busy slots\ + \ per daemon for LLAP). Not used by TezAM (demand-based:\ + \ 1 TezAM per session)." type: integer - targetCpuValue: - description: "Target CPU average value for scaling (e.g.,\ - \ '1500m' or '1'). If omitted, CPU scaling is disabled." - type: string type: object configOverrides: additionalProperties: @@ -564,18 +513,9 @@ spec: description: Tez Application Master configuration. Enabled by default. properties: autoscaling: - description: Autoscaling configuration (requires KEDA installed - in the cluster) + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" properties: - activationCpuValue: - description: CPU average value below which the trigger is - inactive. Required if targetCpuValue is set. - type: string - cooldownSeconds: - default: 600 - description: Cooldown period in seconds after all KEDA triggers - are inactive before scaling from 1 to 0 (scale-to-zero delay) - type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -589,43 +529,35 @@ spec: type: integer metricsScrapeIntervalSeconds: default: 10 - description: Prometheus scrape interval in seconds for this - component's metrics. Lower values make autoscaling react - faster but increase Prometheus load. + description: How often (seconds) the operator scrapes JMX + metrics from pods. Lower values make autoscaling react faster. type: integer minReplicas: default: 0 - description: Minimum number of replicas (floor for scale-down). - Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on - for wake-from-zero) + description: "Minimum number of replicas (floor for scale-down).\ + \ Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum\ + \ is 1)" type: integer scaleDownStabilizationSeconds: - default: 300 + default: 600 description: Stabilization window in seconds for scale-down - decisions. HPA picks the highest recommendation within this - window to prevent premature scale-down. - type: integer - scaleDownThreshold: - default: 20 - description: Threshold that triggers scale-down for Prometheus-based - metrics + decisions. How long metrics must consistently indicate fewer + replicas before scale-down occurs. Also acts as the cooldown + between consecutive scale-downs. type: integer scaleUpStabilizationSeconds: default: 60 description: Stabilization window in seconds for scale-up - decisions. HPA picks the highest recommendation within this + decisions. Picks the highest recommendation within this window to prevent flapping. type: integer scaleUpThreshold: default: 80 description: "Threshold that triggers scale-up (component-specific:\ - \ sessions for HS2, connections for HMS, queue depth for\ - \ LLAP, pending tasks for TezAM)" + \ sessions per pod for HS2, request rate for HMS, busy slots\ + \ per daemon for LLAP). Not used by TezAM (demand-based:\ + \ 1 TezAM per session)." type: integer - targetCpuValue: - description: "Target CPU average value for scaling (e.g.,\ - \ '1500m' or '1'). If omitted, CPU scaling is disabled." - type: string type: object configOverrides: additionalProperties: @@ -732,7 +664,22 @@ spec: type: array hiveServer2: properties: - desiredReplicas: + autoscaling: + properties: + currentMetricValue: + type: integer + lastScaleTime: + type: string + proposedReplicas: + type: integer + scaleUpThreshold: + type: integer + type: object + currentReplicas: + type: integer + maxReplicas: + type: integer + minReplicas: type: integer phase: type: string @@ -741,7 +688,22 @@ spec: type: object llap: properties: - desiredReplicas: + autoscaling: + properties: + currentMetricValue: + type: integer + lastScaleTime: + type: string + proposedReplicas: + type: integer + scaleUpThreshold: + type: integer + type: object + currentReplicas: + type: integer + maxReplicas: + type: integer + minReplicas: type: integer phase: type: string @@ -750,7 +712,22 @@ spec: type: object metastore: properties: - desiredReplicas: + autoscaling: + properties: + currentMetricValue: + type: integer + lastScaleTime: + type: string + proposedReplicas: + type: integer + scaleUpThreshold: + type: integer + type: object + currentReplicas: + type: integer + maxReplicas: + type: integer + minReplicas: type: integer phase: type: string @@ -761,7 +738,22 @@ spec: type: integer tezAm: properties: - desiredReplicas: + autoscaling: + properties: + currentMetricValue: + type: integer + lastScaleTime: + type: string + proposedReplicas: + type: integer + scaleUpThreshold: + type: integer + type: object + currentReplicas: + type: integer + maxReplicas: + type: integer + minReplicas: type: integer phase: type: string diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml index d3df4a5a7868..ba189be3dd2b 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml @@ -30,6 +30,10 @@ rules: - apiGroups: ["apps"] resources: ["deployments", "statefulsets"] verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] + # Scale subresource for operator-driven autoscaling + - apiGroups: ["apps"] + resources: ["deployments/scale", "statefulsets/scale"] + verbs: ["get", "update", "patch"] # Jobs for schema initialization - apiGroups: ["batch"] resources: ["jobs"] @@ -54,11 +58,3 @@ rules: - apiGroups: ["policy"] resources: ["poddisruptionbudgets"] verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] - # KEDA ScaledObjects for autoscaling - - apiGroups: ["keda.sh"] - resources: ["scaledobjects", "triggerauthentications"] - verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] - # KEDA HTTP Add-on for scale-to-zero (wake-from-zero on HTTP request) - - apiGroups: ["http.keda.sh"] - resources: ["httpscaledobjects", "interceptorroutes"] - verbs: ["get", "list", "watch", "create", "update", "patch", "delete"] diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index 9ed5269db04c..f6178ab73a72 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -72,17 +72,10 @@ spec: enabled: true minReplicas: {{ .Values.cluster.metastore.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.metastore.autoscaling.scaleUpThreshold }} - scaleDownThreshold: {{ .Values.cluster.metastore.autoscaling.scaleDownThreshold }} - {{- if .Values.cluster.metastore.autoscaling.targetCpuValue }} - targetCpuValue: {{ .Values.cluster.metastore.autoscaling.targetCpuValue | quote }} - {{- end }} - {{- if .Values.cluster.metastore.autoscaling.activationCpuValue }} - activationCpuValue: {{ .Values.cluster.metastore.autoscaling.activationCpuValue | quote }} - {{- end }} - cooldownSeconds: {{ .Values.cluster.metastore.autoscaling.cooldownSeconds }} scaleUpStabilizationSeconds: {{ .Values.cluster.metastore.autoscaling.scaleUpStabilizationSeconds }} scaleDownStabilizationSeconds: {{ .Values.cluster.metastore.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.metastore.autoscaling.gracePeriodSeconds }} + metricsScrapeIntervalSeconds: {{ .Values.cluster.metastore.autoscaling.metricsScrapeIntervalSeconds | default 10 }} {{- end }} {{- else }} {{- if .Values.cluster.metastore.externalUri }} @@ -118,17 +111,10 @@ spec: enabled: true minReplicas: {{ .Values.cluster.hiveServer2.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleUpThreshold }} - scaleDownThreshold: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownThreshold }} - {{- if .Values.cluster.hiveServer2.autoscaling.targetCpuValue }} - targetCpuValue: {{ .Values.cluster.hiveServer2.autoscaling.targetCpuValue | quote }} - {{- end }} - {{- if .Values.cluster.hiveServer2.autoscaling.activationCpuValue }} - activationCpuValue: {{ .Values.cluster.hiveServer2.autoscaling.activationCpuValue | quote }} - {{- end }} - cooldownSeconds: {{ .Values.cluster.hiveServer2.autoscaling.cooldownSeconds }} scaleUpStabilizationSeconds: {{ .Values.cluster.hiveServer2.autoscaling.scaleUpStabilizationSeconds }} scaleDownStabilizationSeconds: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.hiveServer2.autoscaling.gracePeriodSeconds }} + metricsScrapeIntervalSeconds: {{ .Values.cluster.hiveServer2.autoscaling.metricsScrapeIntervalSeconds | default 10 }} {{- end }} llap: @@ -159,11 +145,10 @@ spec: enabled: true minReplicas: {{ .Values.cluster.llap.autoscaling.minReplicas }} scaleUpThreshold: {{ .Values.cluster.llap.autoscaling.scaleUpThreshold }} - scaleDownThreshold: {{ .Values.cluster.llap.autoscaling.scaleDownThreshold }} - cooldownSeconds: {{ .Values.cluster.llap.autoscaling.cooldownSeconds }} scaleUpStabilizationSeconds: {{ .Values.cluster.llap.autoscaling.scaleUpStabilizationSeconds }} scaleDownStabilizationSeconds: {{ .Values.cluster.llap.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.llap.autoscaling.gracePeriodSeconds }} + metricsScrapeIntervalSeconds: {{ .Values.cluster.llap.autoscaling.metricsScrapeIntervalSeconds | default 10 }} {{- end }} {{- end }} @@ -195,18 +180,10 @@ spec: autoscaling: enabled: true minReplicas: {{ .Values.cluster.tezAm.autoscaling.minReplicas }} - scaleUpThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleUpThreshold }} - scaleDownThreshold: {{ .Values.cluster.tezAm.autoscaling.scaleDownThreshold }} - {{- if .Values.cluster.tezAm.autoscaling.targetCpuValue }} - targetCpuValue: {{ .Values.cluster.tezAm.autoscaling.targetCpuValue | quote }} - {{- end }} - {{- if .Values.cluster.tezAm.autoscaling.activationCpuValue }} - activationCpuValue: {{ .Values.cluster.tezAm.autoscaling.activationCpuValue | quote }} - {{- end }} - cooldownSeconds: {{ .Values.cluster.tezAm.autoscaling.cooldownSeconds }} scaleUpStabilizationSeconds: {{ .Values.cluster.tezAm.autoscaling.scaleUpStabilizationSeconds }} scaleDownStabilizationSeconds: {{ .Values.cluster.tezAm.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.tezAm.autoscaling.gracePeriodSeconds }} + metricsScrapeIntervalSeconds: {{ .Values.cluster.tezAm.autoscaling.metricsScrapeIntervalSeconds | default 10 }} {{- end }} {{- end }} diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index 85bb02c1277f..f122686a9cad 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -112,19 +112,17 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] - # Autoscaling (requires KEDA + Prometheus in the cluster) - # When enabled, 'replicas' above acts as the max replica ceiling + # Autoscaling (operator-driven, no external dependencies) + # The operator scrapes JMX Exporter metrics from pods directly. + # When enabled, 'replicas' above acts as the max replica ceiling. autoscaling: - enabled: false + enabled: true minReplicas: 1 - scaleUpThreshold: 75 - scaleDownThreshold: 30 - # targetCpuValue: "750m" # Uncomment to enable CPU-based scaling (AverageValue) - # activationCpuValue: "200m" # CPU trigger inactive below this value - cooldownSeconds: 300 + scaleUpThreshold: 2 scaleUpStabilizationSeconds: 60 scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 60 + metricsScrapeIntervalSeconds: 10 # Set to use an external Metastore instead of deploying one: # enabled: false # externalUri: "thrift://external-metastore:9083" @@ -140,20 +138,16 @@ cluster: externalJars: [] extraVolumes: [] extraVolumeMounts: [] - # Autoscaling (requires KEDA + Prometheus + KEDA HTTP Add-on in the cluster) - # minReplicas: 0 enables scale-to-zero — beeline HTTP connects wake HS2 via KEDA HTTP interceptor + # Autoscaling (operator-driven, no external dependencies) # When enabled, 'replicas' above acts as the max replica ceiling autoscaling: - enabled: false - minReplicas: 0 - scaleUpThreshold: 80 - scaleDownThreshold: 20 - # targetCpuValue: "1600m" # Uncomment to enable CPU-based scaling (AverageValue) - # activationCpuValue: "400m" # CPU trigger inactive below this value - cooldownSeconds: 600 + enabled: true + minReplicas: 1 + scaleUpThreshold: 1 scaleUpStabilizationSeconds: 60 - scaleDownStabilizationSeconds: 300 + scaleDownStabilizationSeconds: 600 gracePeriodSeconds: 300 + metricsScrapeIntervalSeconds: 10 # --------------------------------------------------------------------------- # LLAP — enabled by default for full-HA @@ -168,18 +162,17 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] - # Autoscaling (requires KEDA + Prometheus in the cluster) - # minReplicas: 0 enables scale-to-zero — scales up immediately when queries need LLAP + # Autoscaling (operator-driven, no external dependencies) + # minReplicas: 0 enables scale-to-zero — scales up when HS2 has active sessions # When enabled, 'replicas' above acts as the max replica ceiling autoscaling: - enabled: false + enabled: true minReplicas: 0 scaleUpThreshold: 1 - scaleDownThreshold: 0 - cooldownSeconds: 900 scaleUpStabilizationSeconds: 60 - scaleDownStabilizationSeconds: 300 + scaleDownStabilizationSeconds: 900 gracePeriodSeconds: 600 + metricsScrapeIntervalSeconds: 10 # --------------------------------------------------------------------------- # TEZ AM — enabled by default for full-HA @@ -193,18 +186,15 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] - # Autoscaling (requires KEDA + Prometheus in the cluster) + # Autoscaling (operator-driven, no external dependencies) # minReplicas: 0 enables scale-to-zero — wakes when HS2 receives queries # When enabled, 'replicas' above acts as the max replica ceiling - # scaleUpThreshold: pending tasks per AM (e.g., 5 = scale when 5+ tasks waiting) + # TezAM scales demand-based: max(totalSessions, hs2Pods * sessionsPerQueue) + # No scaleUpThreshold needed — scaling is 1:1 with session demand autoscaling: - enabled: false + enabled: true minReplicas: 0 - scaleUpThreshold: 5 - scaleDownThreshold: 10 - # targetCpuValue: "600m" # Uncomment to enable CPU-based scaling (AverageValue) - # activationCpuValue: "100m" # CPU trigger inactive below this value - cooldownSeconds: 600 scaleUpStabilizationSeconds: 60 - scaleDownStabilizationSeconds: 300 + scaleDownStabilizationSeconds: 600 gracePeriodSeconds: 120 + metricsScrapeIntervalSeconds: 10 diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java new file mode 100644 index 000000000000..ca435d5f92b2 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java @@ -0,0 +1,122 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.time.Duration; +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Per-component autoscaler state. Owns the scaling strategy, + * stabilization windows. + */ +public class ComponentAutoscaler { + + /** Result of an autoscaling evaluation. */ + public record EvaluationResult(int rawMetricValue, int proposedReplicas, Integer patchTo) {} + + + private static final Logger LOG = LoggerFactory.getLogger(ComponentAutoscaler.class); + + private final String component; + private final ScalingStrategy strategy; + private StabilizationWindow scaleUpWindow; + private StabilizationWindow scaleDownWindow; + private int lastScaleUpStabilization = -1; + private int lastScaleDownStabilization = -1; + private boolean initialized; + + public ComponentAutoscaler(String component, ScalingStrategy strategy) { + this.component = component; + this.strategy = strategy; + } + + /** Whether the underlying strategy uses scaleUpThreshold for scaling decisions. */ + public boolean usesScaleUpThreshold() { + return strategy.usesScaleUpThreshold(); + } + + /** + * Evaluate metrics and return the evaluation result containing + * raw metric value, proposed replicas, and the actual patch (null if no change). + */ + public EvaluationResult evaluate(List metrics, AutoscalingSpec spec, + int currentReplicas, int maxReplicas) { + + ensureWindows(spec); + + // On first evaluation, seed the scale-down window with currentReplicas. + // This prevents immediate scale-down after operator restart when the window has no history. + if (!initialized) { + initialized = true; + scaleDownWindow.record(currentReplicas); + LOG.debug("[{}] Initialized scale-down window with currentReplicas={}", component, currentReplicas); + } + + int rawDesired = strategy.computeDesiredReplicas(metrics, spec, maxReplicas); + int metricValue = strategy.lastMetricValue(); + int clamped = Math.max(spec.minReplicas(), Math.min(rawDesired, maxReplicas)); + + scaleUpWindow.record(clamped); + scaleDownWindow.record(clamped); + + int target; + if (clamped > currentReplicas) { + // Scale up: use stabilized max (highest recommendation in window — don't under-scale) + target = scaleUpWindow.stabilizedMax(); + } else if (clamped < currentReplicas) { + // Scale down: use stabilized max (highest/most conservative recommendation in window — + // prevents premature scale-down, matches HPA selectPolicy: Max behavior). + // The stabilization window duration serves as the cooldown between scale-downs. + target = scaleDownWindow.stabilizedMax(); + } else { + target = currentReplicas; + } + + // Ensure target is still within bounds + target = Math.max(spec.minReplicas(), Math.min(target, maxReplicas)); + + if (target == currentReplicas) { + return new EvaluationResult(metricValue, clamped, null); + } + + if (target < currentReplicas) { + LOG.info("[{}] Scaling down: {} -> {}", component, currentReplicas, target); + } else { + LOG.info("[{}] Scaling up: {} -> {}", component, currentReplicas, target); + } + return new EvaluationResult(metricValue, clamped, target); + } + + private void ensureWindows(AutoscalingSpec spec) { + if (scaleUpWindow == null || lastScaleUpStabilization != spec.scaleUpStabilizationSeconds()) { + scaleUpWindow = new StabilizationWindow( + Duration.ofSeconds(spec.scaleUpStabilizationSeconds())); + lastScaleUpStabilization = spec.scaleUpStabilizationSeconds(); + } + if (scaleDownWindow == null || lastScaleDownStabilization != spec.scaleDownStabilizationSeconds()) { + scaleDownWindow = new StabilizationWindow( + Duration.ofSeconds(spec.scaleDownStabilizationSeconds())); + lastScaleDownStabilization = spec.scaleDownStabilizationSeconds(); + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java new file mode 100644 index 000000000000..5abfc710f7b8 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -0,0 +1,207 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.time.Instant; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import io.fabric8.kubernetes.client.KubernetesClient; +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; +import org.apache.hive.kubernetes.operator.util.Labels; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Main autoscaling orchestrator. Evaluates all enabled components and + * returns a map of component → desired replica count for those that need changing. + *

+ * Maintains per-cluster, per-component state (stabilization windows). + */ +public class HiveClusterAutoscaler { + + private static final Logger LOG = LoggerFactory.getLogger(HiveClusterAutoscaler.class); + + /** Result of evaluating all components. */ + public record AutoscalingEvaluation( + Map patches, + Map statuses) {} + + // Shared replica store: the autoscaler writes its desired replicas here so that + // dependent resources can read them (avoids informer cache lag reverting patches). + // Key: "namespace/clusterName/component" → desired replicas + private static final ConcurrentHashMap MANAGED_REPLICAS = + new ConcurrentHashMap<>(); + + /** + * Returns the autoscaler-managed replica count for a component, or null if the + * autoscaler hasn't made a decision yet (e.g., first reconcile before evaluation runs). + */ + public static Integer getManagedReplicas(String namespace, String clusterName, String component) { + return MANAGED_REPLICAS.get(namespace + "/" + clusterName + "/" + component); + } + + private final MetricsScraper scraper; + // Key: "namespace/clusterName/component" + private final ConcurrentHashMap autoscalers = + new ConcurrentHashMap<>(); + private final ConcurrentHashMap lastScaleTimes = + new ConcurrentHashMap<>(); + + public HiveClusterAutoscaler(MetricsScraper scraper) { + this.scraper = scraper; + } + + /** + * Evaluate all autoscaling-enabled components and return patches and status info. + * + * @param cluster the HiveCluster resource + * @param client the Kubernetes client (for reading current replica counts) + * @return evaluation result with patches and per-component autoscaling statuses + */ + public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient client) { + Map patches = new HashMap<>(); + Map statuses = new HashMap<>(); + HiveClusterSpec spec = cluster.getSpec(); + String namespace = cluster.getMetadata().getNamespace(); + String clusterName = cluster.getMetadata().getName(); + + // HiveServer2 + if (spec.hiveServer2().autoscaling().isEnabled()) { + evaluateComponent(cluster, client, namespace, clusterName, + "hiveserver2", spec.hiveServer2().autoscaling(), + spec.hiveServer2().replicas(), patches, statuses); + } + + // Metastore + if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { + evaluateComponent(cluster, client, namespace, clusterName, + "metastore", spec.metastore().autoscaling(), + spec.metastore().replicas(), patches, statuses); + } + + // LLAP + if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { + evaluateComponent(cluster, client, namespace, clusterName, + "llap", spec.llap().autoscaling(), + spec.llap().replicas(), patches, statuses); + } + + // TezAM + if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { + evaluateComponent(cluster, client, namespace, clusterName, + "tezam", spec.tezAm().autoscaling(), + spec.tezAm().replicas(), patches, statuses); + } + + return new AutoscalingEvaluation(patches, statuses); + } + + /** + * Scrape metrics for HS2 pods (used by LLAP/TezAM activation gate). + */ + public List scrapeHs2Metrics(HiveCluster cluster) { + String namespace = cluster.getMetadata().getNamespace(); + Map selector = Labels.selectorForComponent(cluster, "hiveserver2"); + return scraper.scrape(namespace, selector); + } + + private void evaluateComponent(HiveCluster cluster, KubernetesClient client, + String namespace, String clusterName, String component, + AutoscalingSpec autoscaling, int maxReplicas, + Map patches, Map statuses) { + + int currentReplicas = getCurrentReplicas(client, namespace, clusterName, component); + + String key = namespace + "/" + clusterName + "/" + component; + + Map selector = Labels.selectorForComponent(cluster, component); + List metrics = scraper.scrape(namespace, selector); + + // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), + // not their own pod metrics. Allow evaluation even with 0 own pods. + boolean usesHs2Activation = component.equals("llap") || component.equals("tezam"); + + if (metrics.isEmpty() && !usesHs2Activation) { + LOG.debug("[{}] No ready pods to scrape, skipping", component); + MANAGED_REPLICAS.put(key, currentReplicas); + return; + } + + ComponentAutoscaler autoscaler = autoscalers.computeIfAbsent(key, + k -> new ComponentAutoscaler(component, createStrategy(component, cluster))); + + ComponentAutoscaler.EvaluationResult result = + autoscaler.evaluate(metrics, autoscaling, currentReplicas, maxReplicas); + + // Build status + if (result.patchTo() != null) { + lastScaleTimes.put(key, Instant.now().toString()); + } + AutoscalingStatus as = new AutoscalingStatus(); + as.setCurrentMetricValue(result.rawMetricValue()); + // Only show scaleUpThreshold for strategies that use it (TezAM is demand-based, no threshold) + if (autoscaler.usesScaleUpThreshold()) { + as.setScaleUpThreshold(autoscaling.scaleUpThreshold()); + } + as.setProposedReplicas(result.proposedReplicas()); + as.setLastScaleTime(lastScaleTimes.get(key)); + statuses.put(component, as); + + if (result.patchTo() != null) { + int patchValue = result.patchTo(); + patches.put(component, patchValue); + MANAGED_REPLICAS.put(key, patchValue); + } else { + // No change needed — record current replicas as the managed value + MANAGED_REPLICAS.put(key, currentReplicas); + } + } + + private ScalingStrategy createStrategy(String component, HiveCluster cluster) { + return switch (component) { + case "hiveserver2" -> new HiveServer2ScalingStrategy(); + case "metastore" -> new MetastoreScalingStrategy(); + case "llap" -> new LlapScalingStrategy(this, cluster); + case "tezam" -> new TezAmScalingStrategy(this, cluster); + default -> throw new IllegalArgumentException("Unknown component: " + component); + }; + } + + private int getCurrentReplicas(KubernetesClient client, String namespace, + String clusterName, String component) { + String workloadName = clusterName + "-" + component; + if ("llap".equals(component) || "tezam".equals(component)) { + var ss = client.apps().statefulSets() + .inNamespace(namespace).withName(workloadName).get(); + return ss != null && ss.getSpec().getReplicas() != null ? ss.getSpec().getReplicas() : 0; + } else { + var deploy = client.apps().deployments() + .inNamespace(namespace).withName(workloadName).get(); + return deploy != null && deploy.getSpec().getReplicas() != null + ? deploy.getSpec().getReplicas() : 0; + } + } + +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java new file mode 100644 index 000000000000..72239d8e6041 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java @@ -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.hive.kubernetes.operator.autoscaling; + +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; + +/** + * Scaling strategy for HiveServer2. + * desired = ceil(sum(hs2_open_sessions across all pods) / scaleUpThreshold) + * Uses sum() so that each session is counted — prevents premature scale-down + * of pods that still have active sessions. + */ +public class HiveServer2ScalingStrategy implements ScalingStrategy { + + static final String METRIC_OPEN_SESSIONS = "hs2_open_sessions"; + + private int lastMetric; + + @Override + public int computeDesiredReplicas(List podMetrics, + AutoscalingSpec autoscaling, int maxReplicas) { + // HS2 is the cluster entry point — scaling to 0 makes the cluster unreachable. + // Enforce floor of 1 regardless of CRD defaults or user misconfiguration. + int safeMinReplicas = Math.max(1, autoscaling.minReplicas()); + + double totalSessions = 0; + for (PodMetrics pm : podMetrics) { + totalSessions += pm.metrics().getOrDefault(METRIC_OPEN_SESSIONS, 0.0); + } + + lastMetric = (int) totalSessions; + + if (totalSessions <= 0) { + return safeMinReplicas; + } + + int desired = (int) Math.ceil(totalSessions / autoscaling.scaleUpThreshold()); + return Math.max(desired, safeMinReplicas); + } + + @Override + public int lastMetricValue() { + return lastMetric; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java new file mode 100644 index 000000000000..87ed2a430593 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java @@ -0,0 +1,133 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scaling strategy for LLAP daemons. + * Formula: avg(QueuedRequests + Configured - Available) across all pods. + * This represents average "busy slots + queued" per daemon. + * desired = ceil(avg_busy / scaleUpThreshold) + *

+ * Activation gate: only scale if HS2 has open sessions (prevents zombie scaling). + */ +public class LlapScalingStrategy implements ScalingStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(LlapScalingStrategy.class); + + static final String METRIC_QUEUED = "hadoop_llapdaemon_executornumqueuedrequests"; + static final String METRIC_CONFIGURED = "hadoop_llapdaemon_executornumexecutorsconfigured"; + static final String METRIC_AVAILABLE = "hadoop_llapdaemon_executornumexecutorsavailable"; + + private final HiveClusterAutoscaler orchestrator; + private final HiveCluster cluster; + private int lastMetric; + + public LlapScalingStrategy(HiveClusterAutoscaler orchestrator, HiveCluster cluster) { + this.orchestrator = orchestrator; + this.cluster = cluster; + } + + @Override + public int computeDesiredReplicas(List podMetrics, + AutoscalingSpec autoscaling, int maxReplicas) { + + // Activation gate: check if HS2 has any open sessions. + // If scrape returns empty but LLAP has running pods, treat as "unknown" and preserve. + // This prevents spurious scale-to-zero from transient scrape failures after operator restart. + List hs2Metrics = orchestrator.scrapeHs2Metrics(cluster); + Boolean sessionsDetected = detectHs2Sessions(hs2Metrics); + if (sessionsDetected == null && !podMetrics.isEmpty()) { + // HS2 scrape returned no data but LLAP is running — hold current state + LOG.debug("[llap] HS2 scrape returned no pods; preserving LLAP (has {} running pods)", podMetrics.size()); + lastMetric = 0; + return Math.max(1, autoscaling.minReplicas()); + } + if (sessionsDetected == null || !sessionsDetected) { + LOG.debug("[llap] HS2 has no open sessions, scaling to minReplicas"); + lastMetric = 0; + return autoscaling.minReplicas(); + } + + // HS2 has sessions but LLAP has no pods yet — scale up to at least 1 + if (podMetrics.isEmpty()) { + LOG.debug("[llap] HS2 has sessions but LLAP has 0 pods, scaling to 1"); + lastMetric = 0; + return Math.max(1, autoscaling.minReplicas()); + } + + // Compute average busy slots across all LLAP pods + double totalBusy = 0; + int podCount = 0; + for (PodMetrics pm : podMetrics) { + double queued = pm.metrics().getOrDefault(METRIC_QUEUED, 0.0); + double configured = pm.metrics().getOrDefault(METRIC_CONFIGURED, 0.0); + double available = pm.metrics().getOrDefault(METRIC_AVAILABLE, 0.0); + double busy = queued + configured - available; + totalBusy += busy; + podCount++; + } + + double avgBusy = totalBusy / podCount; + lastMetric = (int) Math.round(avgBusy); + + if (avgBusy <= 0) { + // HS2 has sessions (passed activation gate above) but executors are idle between queries. + // Keep at least 1 daemon to avoid flapping: scaling to 0 here would cause immediate + // scale-back-up on the next evaluation when the empty-pod path triggers. + return Math.max(1, autoscaling.minReplicas()); + } + + LOG.debug("[llap] avgBusy={}, threshold={}", String.format("%.2f", avgBusy), + autoscaling.scaleUpThreshold()); + + int desired = (int) Math.ceil(avgBusy / autoscaling.scaleUpThreshold()); + return Math.max(desired, autoscaling.minReplicas()); + } + + @Override + public int lastMetricValue() { + return lastMetric; + } + + /** + * Detect HS2 open sessions. + * @return true if sessions > 0, false if scraped and all 0, null if scrape returned no pods + * (ambiguous — could be transient failure or HS2 genuinely absent) + */ + private Boolean detectHs2Sessions(List hs2Metrics) { + if (hs2Metrics.isEmpty()) { + return null; + } + for (PodMetrics pm : hs2Metrics) { + double sessions = pm.metrics().getOrDefault( + HiveServer2ScalingStrategy.METRIC_OPEN_SESSIONS, 0.0); + if (sessions > 0) { + return true; + } + } + return false; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java new file mode 100644 index 000000000000..dbac58d928fa --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java @@ -0,0 +1,101 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scaling strategy for Hive Metastore. + * HMS uses HTTP transport — connections are per-request (stateless), so + * open_connections is always ~0. Instead we compute API request rate: + * rate = (sum(api_*_total) - previous_sum) / elapsed_seconds. + * desired = ceil(rate / scaleUpThreshold) + */ +public class MetastoreScalingStrategy implements ScalingStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(MetastoreScalingStrategy.class); + private static final String API_COUNTER_PREFIX = "api_"; + private static final String API_COUNTER_SUFFIX = "_total"; + + // Previous scrape state for rate computation + private final ConcurrentHashMap previousCounters = new ConcurrentHashMap<>(); + private long previousTimestampMs = 0; + private int lastMetric; + + @Override + public int computeDesiredReplicas(List podMetrics, + AutoscalingSpec autoscaling, int maxReplicas) { + + // Sum all api_*_total counters across all pods + double currentTotal = 0; + for (PodMetrics pm : podMetrics) { + for (Map.Entry entry : pm.metrics().entrySet()) { + String name = entry.getKey(); + if (name.startsWith(API_COUNTER_PREFIX) && name.endsWith(API_COUNTER_SUFFIX)) { + currentTotal += entry.getValue(); + } + } + } + + long now = System.currentTimeMillis(); + double rate = 0; + + if (previousTimestampMs > 0) { + double elapsedSeconds = (now - previousTimestampMs) / 1000.0; + if (elapsedSeconds > 0) { + double previousTotal = previousCounters.values().stream() + .mapToDouble(Double::doubleValue).sum(); + double delta = currentTotal - previousTotal; + if (delta < 0) { + // Counter reset (pod restart) — skip this sample + delta = 0; + } + rate = delta / elapsedSeconds; + } + } + + // Store current state for next evaluation + previousCounters.clear(); + previousCounters.put("_total", currentTotal); + previousTimestampMs = now; + + lastMetric = (int) Math.round(rate); + + if (rate <= 0) { + return autoscaling.minReplicas(); + } + + LOG.debug("[metastore] API request rate: {}/s, threshold: {}", + String.format("%.2f", rate), autoscaling.scaleUpThreshold()); + + int desired = (int) Math.ceil(rate / autoscaling.scaleUpThreshold()); + return Math.max(desired, autoscaling.minReplicas()); + } + + @Override + public int lastMetricValue() { + return lastMetric; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java new file mode 100644 index 000000000000..c60e9f96d23c --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java @@ -0,0 +1,142 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.client.KubernetesClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scrapes Prometheus-format metrics from JMX Exporter endpoints on pods. + * Uses pod IPs directly (no Service or Prometheus intermediary). + * All pods are scraped concurrently to avoid blocking the reconciler loop. + */ +public class MetricsScraper { + + private static final Logger LOG = LoggerFactory.getLogger(MetricsScraper.class); + private static final int JMX_EXPORTER_PORT = 9404; + private static final Duration TIMEOUT = Duration.ofSeconds(5); + + private final KubernetesClient client; + private final HttpClient httpClient; + + public MetricsScraper(KubernetesClient client) { + this.client = client; + this.httpClient = HttpClient.newBuilder() + .connectTimeout(TIMEOUT) + .build(); + } + + /** + * Scrape metrics from all ready pods matching the given label selector. + * Pods are scraped concurrently — total wall-clock time is bounded by + * a single pod's timeout (5s) regardless of pod count. + * + * @param namespace the namespace to query + * @param selector label selector (e.g., app.kubernetes.io/component=hiveserver2) + * @return list of per-pod metrics (empty if no pods or all fail) + */ + public List scrape(String namespace, Map selector) { + List pods; + try { + pods = client.pods() + .inNamespace(namespace) + .withLabels(selector) + .list() + .getItems(); + } catch (Exception e) { + LOG.warn("Failed to list pods in {}/{}: {}", namespace, selector, e.getMessage()); + return Collections.emptyList(); + } + + // Filter to ready pods with IPs + List scrapeable = new ArrayList<>(); + for (Pod pod : pods) { + if (isPodReady(pod) && pod.getStatus().getPodIP() != null + && !pod.getStatus().getPodIP().isEmpty()) { + scrapeable.add(pod); + } + } + + if (scrapeable.isEmpty()) { + return Collections.emptyList(); + } + + // Scrape all pods concurrently + List> futures = new ArrayList<>(scrapeable.size()); + for (Pod pod : scrapeable) { + String podName = pod.getMetadata().getName(); + String podIp = pod.getStatus().getPodIP(); + futures.add(fetchMetricsAsync(podIp) + .thenApply(body -> new PodMetrics(podName, PrometheusTextParser.parse(body))) + .exceptionally(ex -> { + LOG.warn("Failed to scrape metrics from pod {}: {}", podName, ex.getMessage()); + return null; + })); + } + + // Wait for all to complete (bounded by TIMEOUT per pod, but all run in parallel) + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join(); + + List results = new ArrayList<>(); + for (CompletableFuture f : futures) { + PodMetrics pm = f.join(); + if (pm != null) { + results.add(pm); + } + } + return results; + } + + private CompletableFuture fetchMetricsAsync(String podIp) { + URI uri = URI.create("http://" + podIp + ":" + JMX_EXPORTER_PORT + "/metrics"); + HttpRequest request = HttpRequest.newBuilder() + .uri(uri) + .timeout(TIMEOUT) + .GET() + .build(); + return httpClient.sendAsync(request, HttpResponse.BodyHandlers.ofString()) + .thenApply(response -> { + if (response.statusCode() != 200) { + throw new RuntimeException("HTTP " + response.statusCode() + " from " + uri); + } + return response.body(); + }); + } + + private static boolean isPodReady(Pod pod) { + if (pod.getStatus() == null || pod.getStatus().getConditions() == null) { + return false; + } + return pod.getStatus().getConditions().stream() + .anyMatch(c -> "Ready".equals(c.getType()) && "True".equals(c.getStatus())); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PodMetrics.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PodMetrics.java new file mode 100644 index 000000000000..1fe54e98e1b3 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PodMetrics.java @@ -0,0 +1,25 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.Map; + +/** Metrics scraped from a single pod's JMX Exporter endpoint. */ +public record PodMetrics(String podName, Map metrics) { +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java new file mode 100644 index 000000000000..88f6dce6fac8 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -0,0 +1,128 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.HashMap; +import java.util.Map; + +/** + * Parses Prometheus text exposition format (from JMX Exporter /metrics). + * Only extracts metric name → value pairs; labels are stripped. + * For metrics with labels, the full line (name + labels) is used as key. + */ +public final class PrometheusTextParser { + + private PrometheusTextParser() { + } + + /** + * Parse Prometheus text format into metric-name → value map. + * Lines with labels are keyed as "metric_name{labels}" to preserve identity. + * Duplicate metric names (e.g. from multiple label sets) are summed. + */ + public static Map parse(String body) { + Map result = new HashMap<>(); + if (body == null || body.isEmpty()) { + return result; + } + for (String line : body.split("\n")) { + if (line.isEmpty() || line.charAt(0) == '#') { + continue; + } + // Format: metric_name[{labels}] value [timestamp] + // We extract metric_name (without labels) and value. + String metricKey; + String valuePart; + int braceStart = line.indexOf('{'); + if (braceStart >= 0) { + int braceEnd = line.indexOf('}', braceStart); + if (braceEnd < 0) { + continue; + } + metricKey = line.substring(0, braceStart); + valuePart = line.substring(braceEnd + 1).trim(); + } else { + int spaceIdx = line.indexOf(' '); + if (spaceIdx < 0) { + continue; + } + metricKey = line.substring(0, spaceIdx); + valuePart = line.substring(spaceIdx + 1).trim(); + } + // valuePart may contain "value timestamp" — take only value + int spaceInValue = valuePart.indexOf(' '); + if (spaceInValue > 0) { + valuePart = valuePart.substring(0, spaceInValue); + } + try { + double value = Double.parseDouble(valuePart); + // Sum duplicates (multiple label sets for same metric name) + result.merge(metricKey, value, Double::sum); + } catch (NumberFormatException e) { + // Skip NaN, +Inf, -Inf, or malformed values + } + } + return result; + } + + /** + * Parse and return per-label-set metrics (preserving labels in key). + * Key format: "metric_name{label=value,...}" + */ + public static Map parseWithLabels(String body) { + Map result = new HashMap<>(); + if (body == null || body.isEmpty()) { + return result; + } + for (String line : body.split("\n")) { + if (line.isEmpty() || line.charAt(0) == '#') { + continue; + } + String metricKey; + String valuePart; + int braceStart = line.indexOf('{'); + if (braceStart >= 0) { + int braceEnd = line.indexOf('}', braceStart); + if (braceEnd < 0) { + continue; + } + metricKey = line.substring(0, braceEnd + 1); + valuePart = line.substring(braceEnd + 1).trim(); + } else { + int spaceIdx = line.indexOf(' '); + if (spaceIdx < 0) { + continue; + } + metricKey = line.substring(0, spaceIdx); + valuePart = line.substring(spaceIdx + 1).trim(); + } + int spaceInValue = valuePart.indexOf(' '); + if (spaceInValue > 0) { + valuePart = valuePart.substring(0, spaceInValue); + } + try { + double value = Double.parseDouble(valuePart); + result.put(metricKey, value); + } catch (NumberFormatException e) { + // Skip + } + } + return result; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ScalingStrategy.java new file mode 100644 index 000000000000..0724fa5f3d9b --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ScalingStrategy.java @@ -0,0 +1,55 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; + +/** Strategy for computing desired replica count from scraped pod metrics. */ +public interface ScalingStrategy { + + /** + * Compute desired replica count based on current pod metrics. + * + * @param podMetrics metrics from all pods of this component + * @param autoscaling the autoscaling configuration + * @param maxReplicas maximum allowed replicas + * @return desired replica count (before stabilization/clamping) + */ + int computeDesiredReplicas(List podMetrics, + AutoscalingSpec autoscaling, int maxReplicas); + + /** + * Returns the raw metric value from the last evaluation (e.g. total sessions, + * request rate, busy slots). Used for status reporting. + */ + default int lastMetricValue() { + return 0; + } + + /** + * Whether this strategy uses scaleUpThreshold from the spec. + * Strategies that are purely demand-based (e.g. TezAM: 1 TezAM per session) + * return false so the threshold is not displayed in status. + */ + default boolean usesScaleUpThreshold() { + return true; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/StabilizationWindow.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/StabilizationWindow.java new file mode 100644 index 000000000000..665b9ac6ef2b --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/StabilizationWindow.java @@ -0,0 +1,74 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayDeque; +import java.util.Deque; + +/** + * HPA-like stabilization window that smooths scaling decisions. + * Keeps a sliding window of desired replica samples and returns + * the max (for scale-up) or min (for scale-down) over the window. + */ +public class StabilizationWindow { + + private record Sample(Instant timestamp, int value) { + } + + private final Deque samples = new ArrayDeque<>(); + private final Duration window; + + public StabilizationWindow(Duration window) { + this.window = window; + } + + /** Record a new desired replica sample. */ + public void record(int desiredReplicas) { + Instant now = Instant.now(); + evictExpired(now); + samples.addLast(new Sample(now, desiredReplicas)); + } + + /** Returns the maximum value in the window (used for scale-up decisions). */ + public int stabilizedMax() { + evictExpired(Instant.now()); + return samples.stream().mapToInt(Sample::value).max().orElse(0); + } + + /** Returns the minimum value in the window (used for scale-down decisions). */ + public int stabilizedMin() { + evictExpired(Instant.now()); + return samples.stream().mapToInt(Sample::value).min().orElse(0); + } + + /** Returns true if the window has at least one sample. */ + public boolean hasSamples() { + evictExpired(Instant.now()); + return !samples.isEmpty(); + } + + private void evictExpired(Instant now) { + Instant cutoff = now.minus(window); + while (!samples.isEmpty() && samples.peekFirst().timestamp().isBefore(cutoff)) { + samples.pollFirst(); + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java new file mode 100644 index 000000000000..e09b6b9ca2ef --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java @@ -0,0 +1,122 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scaling strategy for Tez Application Master. + * TezAM scaling tracks HS2 session demand: + * - Trigger 1 (concurrent): sum(hs2_open_sessions) — each session may need a TezAM + * - Trigger 2 (pre-warm): count(hs2_pods_with_sessions) * sessions_per_queue + * desired = max(trigger1, trigger2) + *

+ * Activation gate: only scale if HS2 has open sessions. + */ +public class TezAmScalingStrategy implements ScalingStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(TezAmScalingStrategy.class); + + private final HiveClusterAutoscaler orchestrator; + private final HiveCluster cluster; + private int lastMetric; + + public TezAmScalingStrategy(HiveClusterAutoscaler orchestrator, HiveCluster cluster) { + this.orchestrator = orchestrator; + this.cluster = cluster; + } + + @Override + public int computeDesiredReplicas(List podMetrics, + AutoscalingSpec autoscaling, int maxReplicas) { + + List hs2Metrics = orchestrator.scrapeHs2Metrics(cluster); + + // Activation gate: if HS2 scrape returns no data but TezAM has running pods, + // treat as "unknown" and preserve current state to avoid spurious scale-to-zero. + if (hs2Metrics.isEmpty() && !podMetrics.isEmpty()) { + LOG.debug("[tezam] HS2 scrape returned no pods; preserving TezAM (has {} running pods)", podMetrics.size()); + lastMetric = 0; + return Math.max(1, autoscaling.minReplicas()); + } + + double totalSessions = 0; + int podsWithSessions = 0; + for (PodMetrics pm : hs2Metrics) { + double sessions = pm.metrics().getOrDefault( + HiveServer2ScalingStrategy.METRIC_OPEN_SESSIONS, 0.0); + totalSessions += sessions; + if (sessions > 0) { + podsWithSessions++; + } + } + + if (totalSessions <= 0) { + LOG.debug("[tezam] No HS2 sessions, scaling to minReplicas"); + lastMetric = 0; + return autoscaling.minReplicas(); + } + + lastMetric = (int) totalSessions; + + // Trigger 1: concurrent demand — total open sessions (1 TezAM per session) + int concurrentDemand = (int) Math.ceil(totalSessions); + + // Trigger 2: pre-warm — only if hive.server2.tez.initialize.default.sessions is true. + // When true, each HS2 pod pre-warms sessionsPerQueue TezAMs at startup. + // When false, no pre-warming happens — scale purely on concurrent session demand. + int prewarmDemand = 0; + boolean initSessions = ConfigUtils.getBoolean( + cluster.getSpec().hiveServer2().configOverrides(), + ConfigUtils.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_KEY, + ConfigUtils.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_DEFAULT); + if (initSessions) { + int sessionsPerQueue = ConfigUtils.getInt( + cluster.getSpec().hiveServer2().configOverrides(), + ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY, + null, ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT); + prewarmDemand = podsWithSessions * sessionsPerQueue; + } + + int desired = Math.max(concurrentDemand, prewarmDemand); + + LOG.debug("[tezam] totalSessions={}, podsWithSessions={}, initDefaultSessions={}, " + + "concurrent={}, prewarm={}, desired={}", + totalSessions, podsWithSessions, initSessions, + concurrentDemand, prewarmDemand, desired); + + return Math.max(desired, autoscaling.minReplicas()); + } + + @Override + public int lastMetricValue() { + return lastMetric; + } + + @Override + public boolean usesScaleUpThreshold() { + return false; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 8683c06d7bb2..caa7f17dc340 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -45,6 +45,7 @@ import io.javaoperatorsdk.operator.api.reconciler.Context; import io.javaoperatorsdk.operator.processing.dependent.Matcher; import io.javaoperatorsdk.operator.processing.dependent.kubernetes.CRUDKubernetesDependentResource; +import org.apache.hive.kubernetes.operator.autoscaling.HiveClusterAutoscaler; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.DatabaseConfig; @@ -162,32 +163,55 @@ protected R handleCreate(R desired, P primary, Context

context) { /** * Resolves the replica count to set in the desired workload spec. - * When autoscaling is enabled and the workload already exists, the current - * replica count is preserved (KEDA/HPA manages it). On initial creation - * the provided fallback is used. - * - * @param primary the HiveCluster primary resource - * @param context the reconciliation context - * @param autoscaling autoscaling spec for this component (may be null) - * @param staticReplicas replica count from the spec (used when autoscaling is off) - * @param initialReplicas replica count on first creation when autoscaling is on + *

+ * Always returns an explicit value — never null. Returning null would cause + * JOSDK/SSA to omit spec.replicas, and Kubernetes would default it to 1. + *

+ * When autoscaling is enabled: + * - On CREATE: returns initialReplicas (minReplicas for the component) + * - On UPDATE: returns the autoscaler's managed value, or falls back to + * the current actual replicas from the informer cache. + *

+ * When autoscaling is disabled: returns staticReplicas (the spec value). */ - @SuppressWarnings("unchecked") protected Integer resolveReplicaCount(P primary, Context

context, AutoscalingSpec autoscaling, int staticReplicas, int initialReplicas) { if (autoscaling == null || !autoscaling.isEnabled()) { return staticReplicas; } - return getSecondaryResource(primary, context) - .map(existing -> { - if (existing instanceof io.fabric8.kubernetes.api.model.apps.Deployment d) { - return d.getSpec().getReplicas(); - } else if (existing instanceof io.fabric8.kubernetes.api.model.apps.StatefulSet s) { - return s.getSpec().getReplicas(); - } - return initialReplicas; - }) - .orElse(initialReplicas); + Optional existing = getSecondaryResource(primary, context); + if (existing.isPresent()) { + // Check if the autoscaler has made a decision during this operator's lifecycle + Integer managed = HiveClusterAutoscaler.getManagedReplicas( + primary.getMetadata().getNamespace(), + primary.getMetadata().getName(), + getComponentName()); + if (managed != null) { + return managed; + } + // Fallback: operator restarted and MANAGED_REPLICAS is empty — read current value + R resource = existing.get(); + if (resource instanceof io.fabric8.kubernetes.api.model.apps.Deployment d) { + return d.getSpec() != null && d.getSpec().getReplicas() != null + ? d.getSpec().getReplicas() : initialReplicas; + } + if (resource instanceof io.fabric8.kubernetes.api.model.apps.StatefulSet s) { + return s.getSpec() != null && s.getSpec().getReplicas() != null + ? s.getSpec().getReplicas() : initialReplicas; + } + return initialReplicas; + } + // First creation: start at minReplicas. + return initialReplicas; + } + + + /** + * Returns the component name for this dependent (used for autoscaler replica lookup). + * Subclasses should override if they manage a workload with autoscaling. + */ + protected String getComponentName() { + return null; } /** @@ -803,7 +827,7 @@ private static String buildJmxExporterConfig(String component) { sb.append(" type: GAUGE\n"); break; case "llap": - // Only export the executor metrics KEDA and the drain script need. + // Only export the executor metrics the autoscaler and drain script need. // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, // causing CPU spikes and GC pressure on the LLAP JVM. // Internal format: Hadoop><>Attribute diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java deleted file mode 100644 index de9fb6351824..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveGenericDependentResource.java +++ /dev/null @@ -1,209 +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.hive.kubernetes.operator.dependent; - -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; - - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.javaoperatorsdk.operator.api.config.informer.InformerEventSourceConfiguration; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.api.reconciler.EventSourceContext; -import io.javaoperatorsdk.operator.api.reconciler.dependent.GarbageCollected; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import io.javaoperatorsdk.operator.processing.dependent.Creator; -import io.javaoperatorsdk.operator.processing.dependent.Updater; -import io.javaoperatorsdk.operator.processing.dependent.kubernetes.GenericKubernetesDependentResource; -import org.apache.hive.kubernetes.operator.model.HiveCluster; - -/** - * Base class for dependent resources that manage custom resources via - * {@link GenericKubernetesResource} (e.g. KEDA ScaledObject, HTTPScaledObject). - *

- * Extends {@link GenericKubernetesDependentResource} which properly configures - * the informer with the specified GroupVersionKind, avoiding the fabric8 - * "resources cannot be called with a generic type" error. - *

- * Also overrides {@link #getSecondaryResource} to use the dependent's own - * event source (same pattern as {@link HiveDependentResource}) so multiple - * GenericKubernetesResource dependents don't collide in the type-based lookup. - */ -public abstract class HiveGenericDependentResource - extends GenericKubernetesDependentResource - implements Creator, - Updater, - GarbageCollected { - - protected HiveGenericDependentResource(GroupVersionKind gvk) { - super(gvk); - } - - /** - * Adds a generation-aware update filter so that KEDA/controller status - * patches (which don't increment metadata.generation) do not trigger - * unnecessary reconciliation loops. - */ - @Override - protected InformerEventSourceConfiguration.Builder - informerConfigurationBuilder(EventSourceContext context) { - return super.informerConfigurationBuilder(context) - .withOnUpdateFilter((newResource, oldResource) -> { - Long newGen = newResource.getMetadata().getGeneration(); - Long oldGen = oldResource.getMetadata().getGeneration(); - return !Objects.equals(newGen, oldGen); - }); - } - - /** - * Returns the expected Kubernetes resource name for this dependent given the primary. - * Used to discriminate between multiple secondary resources of the same GVK - * (e.g. multiple ScaledObjects owned by the same HiveCluster). - */ - protected abstract String getResourceName(HiveCluster hiveCluster); - - @Override - public Optional getSecondaryResource( - HiveCluster primary, Context context) { - String expectedName = getResourceName(primary); - Set secondaries = eventSource() - .map(es -> es.getSecondaryResources(primary)) - .orElse(Set.of()); - return secondaries.stream() - .filter(r -> expectedName.equals(r.getMetadata().getName())) - .findFirst(); - } - - /** - * Builds the nested "advanced" HPA behavior configuration for a KEDA ScaledObject. - * - * @param scaleDownStabilization stabilizationWindowSeconds for scale-down - * @param scaleDownPolicyType policy type (e.g. "Pods", "Percent") - * @param scaleDownValue policy value - * @param scaleDownPeriod policy periodSeconds - * @param scaleUpStabilization stabilizationWindowSeconds for scale-up - * @param scaleUpPolicyType policy type (e.g. "Pods", "Percent") - * @param scaleUpValue policy value - * @param scaleUpPeriod policy periodSeconds - */ - protected static Map buildHpaBehavior( - int scaleDownStabilization, String scaleDownPolicyType, - int scaleDownValue, int scaleDownPeriod, - int scaleUpStabilization, String scaleUpPolicyType, - int scaleUpValue, int scaleUpPeriod) { - return Map.of( - "horizontalPodAutoscalerConfig", Map.of( - "behavior", Map.of( - "scaleDown", Map.of( - "stabilizationWindowSeconds", scaleDownStabilization, - "policies", List.of(Map.of( - "type", scaleDownPolicyType, - "value", scaleDownValue, - "periodSeconds", scaleDownPeriod - )) - ), - "scaleUp", Map.of( - "stabilizationWindowSeconds", scaleUpStabilization, - "policies", List.of(Map.of( - "type", scaleUpPolicyType, - "value", scaleUpValue, - "periodSeconds", scaleUpPeriod - )) - ) - ) - ) - ); - } - - /** - * Builds the HS2 cross-component activation trigger used by LLAP and TezAM. - * Uses {@code (max(hs2_open_sessions{...}) > bool 0) or vector(0)} so the - * result is always 0 or 1, preventing zombie sessions from driving proportional scaling. - * Threshold is set to maxReplicas so desired = ceil(1/max) = 1 (activation only). - * - * @param namespace the Kubernetes namespace - * @param hs2TargetName the HS2 deployment name (for pod label matching) - * @param maxReplicas the max replicas of the component (used as threshold) - */ - protected static Map buildHs2ActivationTrigger( - String namespace, String hs2TargetName, int maxReplicas) { - return buildPrometheusTrigger( - "hs2_open_sessions_activation", - String.format( - "(max(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) > bool 0) or vector(0)", - namespace, hs2TargetName), - String.valueOf(maxReplicas)); - } - - /** - * Builds a KEDA Prometheus trigger entry. - * - * @param metricName the KEDA metric name - * @param query the PromQL query - * @param threshold the scaling threshold value - */ - protected static Map buildPrometheusTrigger( - String metricName, String query, String threshold) { - return Map.of( - "type", "prometheus", - "metadata", Map.of( - "serverAddress", "http://prometheus-server.monitoring.svc.cluster.local", - "metricName", metricName, - "query", query, - "threshold", threshold, - "activationThreshold", "0" - ) - ); - } - - /** - * Builds a KEDA CPU AverageValue trigger if both targetCpuValue and - * activationCpuValue are configured. Returns null if CPU scaling is - * not configured, or if resources are missing (logs a warning). - * - * @param autoscaling the autoscaling spec - * @param resources the pod resource spec (null means not set) - * @param componentName component name for the warning message - * @param log the logger to use for warnings - */ - protected static Map buildCpuTrigger( - org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec autoscaling, - Object resources, String componentName, - org.slf4j.Logger log) { - if (autoscaling.targetCpuValue() == null || autoscaling.activationCpuValue() == null) { - return null; - } - if (resources == null) { - log.warn("targetCpuValue is set for {}, but no pod resources are defined. " - + "Skipping CPU trigger to prevent erratic scaling.", componentName); - return null; - } - return Map.of( - "type", "cpu", - "metricType", "AverageValue", - "metadata", Map.of( - "value", autoscaling.targetCpuValue(), - "activationValue", autoscaling.activationCpuValue() - ) - ); - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java deleted file mode 100644 index 52a3624639ca..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveScaledObjectDependent.java +++ /dev/null @@ -1,373 +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.hive.kubernetes.operator.dependent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Unified KEDA ScaledObject dependent resource for metric-based autoscaling. - * Subclassed per component to define component-specific triggers, HPA behavior, - * and target workload kind. - *

- * Note: When HS2 minReplicas is 0, the ScaledObject includes an external-push - * trigger from the KEDA HTTP Add-on (via InterceptorRoute) for wake-from-zero. - */ -public abstract class HiveScaledObjectDependent extends HiveGenericDependentResource { - - private static final Logger LOG = LoggerFactory.getLogger(HiveScaledObjectDependent.class); - - private final String component; - private final String targetKind; - - protected HiveScaledObjectDependent(String component, String targetKind) { - super(new GroupVersionKind("keda.sh", "v1alpha1", "ScaledObject")); - this.component = component; - this.targetKind = targetKind; - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = getAutoscalingSpec(hiveCluster); - int maxReplicas = getMaxReplicas(hiveCluster); - String targetName = hiveCluster.getMetadata().getName() + "-" + component; - - Map spec = new HashMap<>(); - spec.put("scaleTargetRef", Map.of( - "apiVersion", "apps/v1", - "kind", targetKind, - "name", targetName - )); - spec.put("minReplicaCount", autoscaling.minReplicas()); - spec.put("maxReplicaCount", maxReplicas); - spec.put("cooldownPeriod", autoscaling.cooldownSeconds()); - spec.put("pollingInterval", getPollingInterval()); - spec.put("advanced", getAdvanced(hiveCluster, autoscaling, maxReplicas)); - spec.put("triggers", getTriggers(hiveCluster, autoscaling, maxReplicas, targetName)); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("keda.sh/v1alpha1") - .withKind("ScaledObject") - .withNewMetadata() - .withName(targetName + "-scaledobject") - .withNamespace(hiveCluster.getMetadata().getNamespace()) - .withLabels(Labels.forComponent(hiveCluster, component)) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-" + component + "-scaledobject"; - } - - /** Returns the autoscaling spec for the component. */ - protected abstract AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster); - - /** Returns max replicas (typically the static replicas count from spec). */ - protected abstract int getMaxReplicas(HiveCluster hiveCluster); - - /** Returns the KEDA polling interval in seconds. */ - protected abstract int getPollingInterval(); - - /** Returns the "advanced" section (HPA behavior configuration). */ - protected abstract Map getAdvanced( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas); - - /** Returns the list of KEDA triggers. */ - protected abstract List> getTriggers( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, - int maxReplicas, String targetName); - - /** - * HiveServer2 ScaledObject: scales on hs2_active_sessions + CPU. - */ - public static class HiveServer2 extends HiveScaledObjectDependent { - public HiveServer2() { - super("hiveserver2", "Deployment"); - } - - @Override - protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { - return hiveCluster.getSpec().hiveServer2().autoscaling(); - } - - @Override - protected int getMaxReplicas(HiveCluster hiveCluster) { - return hiveCluster.getSpec().hiveServer2().replicas(); - } - - @Override - protected int getPollingInterval() { - return 30; - } - - @Override - protected Map getAdvanced( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { - return buildHpaBehavior( - autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, - autoscaling.scaleUpStabilizationSeconds(), "Percent", 100, 60); - } - - @Override - protected List> getTriggers( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, - int maxReplicas, String targetName) { - List> triggers = new ArrayList<>(); - // Use sum() so KEDA computes desired replicas from total session count. - // desired = ceil(sum / threshold). With sum=2, threshold=1: desired=2 - // → prevents premature scale-down while sessions are active. - // avg() would divide across pods, hiding load and causing scale-down - // of pods with sessions. - triggers.add(buildPrometheusTrigger( - "hs2_open_sessions", - String.format( - "sum(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName), - String.valueOf(autoscaling.scaleUpThreshold()))); - Map cpuTrigger = buildCpuTrigger( - autoscaling, hiveCluster.getSpec().hiveServer2().resources(), - "HiveServer2", LOG); - if (cpuTrigger != null) { - triggers.add(cpuTrigger); - } - // When scale-to-zero is enabled, add KEDA HTTP Add-on external-push - // trigger to wake HS2 from 0 when requests arrive at the interceptor. - if (autoscaling.minReplicas() == 0) { - String routeName = HiveServer2InterceptorRouteDependent.resourceName(hiveCluster); - triggers.add(Map.of( - "type", "external-push", - "metadata", Map.of( - "scalerAddress", - "keda-add-ons-http-external-scaler.keda:9090", - "interceptorRoute", routeName - ) - )); - } - return triggers; - } - } - - /** - * Metastore ScaledObject: scales on open_connections + CPU. - */ - public static class Metastore extends HiveScaledObjectDependent { - public Metastore() { - super("metastore", "Deployment"); - } - - @Override - protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { - return hiveCluster.getSpec().metastore().autoscaling(); - } - - @Override - protected int getMaxReplicas(HiveCluster hiveCluster) { - return hiveCluster.getSpec().metastore().replicas(); - } - - @Override - protected int getPollingInterval() { - return 30; - } - - @Override - protected Map getAdvanced( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { - return buildHpaBehavior( - autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, - autoscaling.scaleUpStabilizationSeconds(), "Percent", 50, 60); - } - - @Override - protected List> getTriggers( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, - int maxReplicas, String targetName) { - List> triggers = new ArrayList<>(); - // HMS runs in HTTP transport mode — connections are per-request (stateless), - // so open_connections is always ~0. Use aggregate API request rate instead. - // Note: Prometheus 3.x rejects rate() on __name__ regex selectors, so we - // compute rate manually as (sum(counters) - sum(counters offset 2m)) / 120. - triggers.add(buildPrometheusTrigger( - "hive_metastore_api_rate", - String.format( - "(sum({__name__=~\"api_.+_total\",namespace=\"%s\",pod=~\"%s-.*\"})" - + " - sum({__name__=~\"api_.+_total\",namespace=\"%s\",pod=~\"%s-.*\"} offset 2m))" - + " / 120 or vector(0)", - hiveCluster.getMetadata().getNamespace(), targetName, - hiveCluster.getMetadata().getNamespace(), targetName), - String.valueOf(autoscaling.scaleUpThreshold()))); - Map cpuTrigger = buildCpuTrigger( - autoscaling, hiveCluster.getSpec().metastore().resources(), - "Metastore", LOG); - if (cpuTrigger != null) { - triggers.add(cpuTrigger); - } - return triggers; - } - } - - /** - * LLAP ScaledObject: scales on NumQueuedRequests + HS2 activation trigger. - * Scale-down is slow (preserves in-memory cache). - */ - public static class Llap extends HiveScaledObjectDependent { - public Llap() { - super("llap", "StatefulSet"); - } - - @Override - protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { - return hiveCluster.getSpec().llap().autoscaling(); - } - - @Override - protected int getMaxReplicas(HiveCluster hiveCluster) { - return hiveCluster.getSpec().llap().replicas(); - } - - @Override - protected int getPollingInterval() { - return 5; - } - - @Override - protected Map getAdvanced( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { - // Scale-up stabilization=0: LLAP is a reactive dependent that must - // track HS2 immediately — no delay on scale-up. - return buildHpaBehavior( - autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, autoscaling.scaleDownStabilizationSeconds(), - 0, "Pods", maxReplicas, 15); - } - - @Override - protected List> getTriggers( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, - int maxReplicas, String targetName) { - String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; - String namespace = hiveCluster.getMetadata().getNamespace(); - return List.of( - buildPrometheusTrigger( - "llap_total_busy_slots", - String.format( - "avg(" - + "hadoop_llapdaemon_executornumqueuedrequests{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + " + on(pod) hadoop_llapdaemon_executornumexecutorsconfigured{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + " - on(pod) hadoop_llapdaemon_executornumexecutorsavailable{namespace=\"%1$s\",pod=~\"%2$s-.*\"}" - + ") or vector(0)", - namespace, targetName), - String.valueOf(autoscaling.scaleUpThreshold())), - buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas) - ); - } - } - - /** - * TezAM ScaledObject: scales on HS2 session demand. - * Each HS2 pod claims {@code sessions.per.default.queue} TezAM sessions - * (exclusive binding). Demand = active HS2 pods × sessions per queue. - * Primary trigger: count of HS2 pods with open sessions × sessions_per_queue. - */ - public static class TezAm extends HiveScaledObjectDependent { - public TezAm() { - super("tezam", "StatefulSet"); - } - - @Override - protected AutoscalingSpec getAutoscalingSpec(HiveCluster hiveCluster) { - return hiveCluster.getSpec().tezAm().autoscaling(); - } - - @Override - protected int getMaxReplicas(HiveCluster hiveCluster) { - return hiveCluster.getSpec().tezAm().replicas(); - } - - @Override - protected int getPollingInterval() { - return 5; - } - - @Override - protected Map getAdvanced( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, int maxReplicas) { - // Scale-up stabilization=0: TezAM is a reactive dependent that must - // track HS2 sessions immediately — no delay on scale-up. - return buildHpaBehavior( - autoscaling.scaleDownStabilizationSeconds(), "Pods", 1, 60, - 0, "Pods", maxReplicas, 15); - } - - @Override - protected List> getTriggers( - HiveCluster hiveCluster, AutoscalingSpec autoscaling, - int maxReplicas, String targetName) { - String hs2TargetName = hiveCluster.getMetadata().getName() + "-hiveserver2"; - String namespace = hiveCluster.getMetadata().getNamespace(); - - // Read sessions.per.default.queue from HS2 configOverrides (default 1). - // Each HS2 pod pre-warms this many TezAM sessions in its pool. - int sessionsPerQueue = ConfigUtils.getInt( - hiveCluster.getSpec().hiveServer2().configOverrides(), - ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY, - null, ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT); - - List> triggers = new ArrayList<>(); - // Trigger 1: Concurrent demand — total open sessions across all HS2 pods. - // Each session may run a query needing its own TezAM. - // threshold=1 → desired = total open sessions. - triggers.add(buildPrometheusTrigger( - "hs2_tezam_session_demand", - String.format( - "sum(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) or vector(0)", - namespace, hs2TargetName), - "1")); - // Trigger 2: Pre-warm — each running HS2 pod needs sessions_per_queue TezAMs - // in its pool (claimed eagerly at startup by default). - // threshold=1 → desired = HS2_pod_count × sessions_per_queue. - triggers.add(buildPrometheusTrigger( - "hs2_tezam_prewarm", - String.format( - "count(hs2_open_sessions{namespace=\"%s\",pod=~\"%s-.*\"}) * %d or vector(0)", - namespace, hs2TargetName, sessionsPerQueue), - "1")); - // KEDA uses max(trigger1, trigger2) → ensures enough TezAMs for both - // concurrent queries AND per-HS2 pre-warm pools. - triggers.add(buildHs2ActivationTrigger(namespace, hs2TargetName, maxReplicas)); - return triggers; - } - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index 20d27a46b0f7..4ce67ae9f6d4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -62,6 +62,11 @@ protected String getSecondaryResourceName(HiveCluster primary, return resourceName(primary); } + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -215,10 +220,9 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). AutoscalingSpec hs2Autoscaling = hs2.autoscaling(); - int initialReplicas = hs2Autoscaling != null && hs2Autoscaling.minReplicas() == 0 - ? 0 : hs2.replicas(); + int initialReplicas = hs2Autoscaling != null && hs2Autoscaling.isEnabled() + ? Math.max(1, hs2Autoscaling.minReplicas()) : hs2.replicas(); Integer replicas = resolveReplicaCount( hiveCluster, context, hs2Autoscaling, hs2.replicas(), initialReplicas); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java deleted file mode 100644 index de6e3bb71d5c..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2InterceptorRouteDependent.java +++ /dev/null @@ -1,116 +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.hive.kubernetes.operator.dependent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import io.fabric8.kubernetes.api.model.GenericKubernetesResource; -import io.fabric8.kubernetes.api.model.GenericKubernetesResourceBuilder; -import io.javaoperatorsdk.operator.api.reconciler.Context; -import io.javaoperatorsdk.operator.processing.GroupVersionKind; -import org.apache.hive.kubernetes.operator.model.HiveCluster; -import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; -import org.apache.hive.kubernetes.operator.util.Labels; - -/** - * Manages a KEDA InterceptorRoute for HiveServer2 scale-to-zero routing. - *

- * Unlike HTTPScaledObject, InterceptorRoute only configures interceptor - * routing without auto-creating a ScaledObject. This allows us to manage - * scaling entirely through a single Prometheus-based ScaledObject that - * combines session/CPU awareness with the HTTP interceptor wake-from-zero - * trigger. - *

- * Requires the KEDA HTTP Add-on to be installed in the cluster. - */ -public class HiveServer2InterceptorRouteDependent extends HiveGenericDependentResource { - - public HiveServer2InterceptorRouteDependent() { - super(new GroupVersionKind("http.keda.sh", "v1beta1", "InterceptorRoute")); - } - - @Override - protected GenericKubernetesResource desired(HiveCluster hiveCluster, - Context context) { - AutoscalingSpec autoscaling = hiveCluster.getSpec().hiveServer2().autoscaling(); - String clusterName = hiveCluster.getMetadata().getName(); - String namespace = hiveCluster.getMetadata().getNamespace(); - String serviceName = clusterName + "-hiveserver2"; - - int httpPort = ConfigUtils.getInt( - hiveCluster.getSpec().hiveServer2().configOverrides(), - ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_KEY, - null, ConfigUtils.HIVE_SERVER2_THRIFT_HTTP_PORT_DEFAULT); - - // Hosts the interceptor matches for routing - List hosts = new ArrayList<>(List.of( - serviceName + "." + namespace + ".svc.cluster.local", - serviceName, - "keda-add-ons-http-interceptor-proxy.keda.svc", - "localhost" - )); - - Map spec = new HashMap<>(); - - // Target backend service - spec.put("target", Map.of( - "service", serviceName, - "port", httpPort - )); - - // Routing rules - spec.put("rules", List.of( - Map.of( - "hosts", hosts, - "paths", List.of(Map.of("value", "/")) - ) - )); - - // Scaling metric (required field, used by interceptor for queue management) - spec.put("scalingMetric", Map.of( - "concurrency", Map.of( - "targetValue", autoscaling.scaleUpThreshold() - ) - )); - - return new GenericKubernetesResourceBuilder() - .withApiVersion("http.keda.sh/v1beta1") - .withKind("InterceptorRoute") - .withNewMetadata() - .withName(resourceName(hiveCluster)) - .withNamespace(namespace) - .withLabels(Labels.forComponent(hiveCluster, "hiveserver2")) - .endMetadata() - .withAdditionalProperties(Map.of("spec", spec)) - .build(); - } - - @Override - protected String getResourceName(HiveCluster hiveCluster) { - return resourceName(hiveCluster); - } - - public static String resourceName(HiveCluster hiveCluster) { - return hiveCluster.getMetadata().getName() + "-hiveserver2-route"; - } -} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 171766e6f341..09e68fbf5e70 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -63,6 +63,11 @@ protected String getSecondaryResourceName(HiveCluster primary, return resourceName(primary); } + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -133,10 +138,9 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getLlapDaemonSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). AutoscalingSpec llapAutoscaling = llap.autoscaling(); - int initialReplicas = llapAutoscaling != null && llapAutoscaling.minReplicas() == 0 - ? 0 : llap.replicas(); + int initialReplicas = llapAutoscaling != null && llapAutoscaling.isEnabled() + ? llapAutoscaling.minReplicas() : llap.replicas(); Integer replicas = resolveReplicaCount( hiveCluster, context, llapAutoscaling, llap.replicas(), initialReplicas); @@ -199,6 +203,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), spec.volumes(), spec.volumeMounts(), llap.extraVolumes(), llap.extraVolumeMounts()); + return statefulSet; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index 34dcc099abcc..ac8ba55df4a4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -63,6 +63,11 @@ protected String getSecondaryResourceName(HiveCluster primary, return resourceName(primary); } + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -128,9 +133,8 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getMetastoreSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). AutoscalingSpec msAutoscaling = spec.metastore().autoscaling(); - int initialReplicas = msAutoscaling != null + int initialReplicas = msAutoscaling != null && msAutoscaling.isEnabled() ? Math.max(1, msAutoscaling.minReplicas()) : spec.metastore().replicas(); Integer replicas = resolveReplicaCount( hiveCluster, context, msAutoscaling, spec.metastore().replicas(), initialReplicas); @@ -194,6 +198,7 @@ protected Deployment desired(HiveCluster hiveCluster, appendUserVolumes(deployment.getSpec().getTemplate().getSpec(), spec.volumes(), spec.volumeMounts(), spec.metastore().extraVolumes(), spec.metastore().extraVolumeMounts()); + return deployment; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index ac83286e346c..3b14cf54b494 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -65,6 +65,11 @@ protected String getSecondaryResourceName(HiveCluster primary, return resourceName(primary); } + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -136,10 +141,9 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); - // When autoscaling is enabled, preserve current replica count (KEDA/HPA manages it). AutoscalingSpec tezAmAutoscaling = tezAm.autoscaling(); - int initialReplicas = tezAmAutoscaling != null && tezAmAutoscaling.minReplicas() == 0 - ? 0 : tezAm.replicas(); + int initialReplicas = tezAmAutoscaling != null && tezAmAutoscaling.isEnabled() + ? tezAmAutoscaling.minReplicas() : tezAm.replicas(); Integer replicas = resolveReplicaCount( hiveCluster, context, tezAmAutoscaling, tezAm.replicas(), initialReplicas); @@ -198,6 +202,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), spec.volumes(), spec.volumeMounts(), tezAm.extraVolumes(), tezAm.extraVolumeMounts()); + return statefulSet; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java index ab02949d7f25..2ec6d5fda0c3 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -21,40 +21,28 @@ import com.fasterxml.jackson.annotation.JsonPropertyDescription; import io.fabric8.generator.annotation.Default; -/** Autoscaling configuration for a Hive component. Uses KEDA ScaledObjects for metric-based scaling. */ +/** Autoscaling configuration for a Hive component. The operator scrapes JMX metrics directly from pods. */ public record AutoscalingSpec( @JsonPropertyDescription("Whether autoscaling is enabled for this component") @Default("false") Boolean enabled, @JsonPropertyDescription("Minimum number of replicas (floor for scale-down). " - + "Set to 0 for scale-to-zero (HS2 requires KEDA HTTP Add-on for wake-from-zero)") + + "Set to 0 for scale-to-zero (LLAP, TezAM only; HS2 minimum is 1)") @Default("0") Integer minReplicas, @JsonPropertyDescription("Threshold that triggers scale-up (component-specific: " - + "sessions for HS2, connections for HMS, queue depth for LLAP, " - + "pending tasks for TezAM)") + + "sessions per pod for HS2, request rate for HMS, busy slots per daemon for LLAP). " + + "Not used by TezAM (demand-based: 1 TezAM per session).") @Default("80") Integer scaleUpThreshold, - @JsonPropertyDescription("Threshold that triggers scale-down for Prometheus-based metrics") - @Default("20") - Integer scaleDownThreshold, - @JsonPropertyDescription("Target CPU average value for scaling (e.g., '1500m' or '1'). " - + "If omitted, CPU scaling is disabled.") - String targetCpuValue, - @JsonPropertyDescription("CPU average value below which the trigger is inactive. " - + "Required if targetCpuValue is set.") - String activationCpuValue, - @JsonPropertyDescription("Cooldown period in seconds after all KEDA triggers are inactive " - + "before scaling from 1 to 0 (scale-to-zero delay)") - @Default("600") - Integer cooldownSeconds, @JsonPropertyDescription("Stabilization window in seconds for scale-up decisions. " - + "HPA picks the highest recommendation within this window to prevent flapping.") + + "Picks the highest recommendation within this window to prevent flapping.") @Default("60") Integer scaleUpStabilizationSeconds, @JsonPropertyDescription("Stabilization window in seconds for scale-down decisions. " - + "HPA picks the highest recommendation within this window to prevent premature scale-down.") - @Default("300") + + "How long metrics must consistently indicate fewer replicas before " + + "scale-down occurs. Also acts as the cooldown between consecutive scale-downs.") + @Default("600") Integer scaleDownStabilizationSeconds, @JsonPropertyDescription("Maximum time in seconds to wait for graceful drain " + "during scale-down before the pod is forcibly terminated. " @@ -62,8 +50,8 @@ public record AutoscalingSpec( + "this value is only the upper safety cap.") @Default("3600") Integer gracePeriodSeconds, - @JsonPropertyDescription("Prometheus scrape interval in seconds for this component's metrics. " - + "Lower values make autoscaling react faster but increase Prometheus load.") + @JsonPropertyDescription("How often (seconds) the operator scrapes JMX metrics from pods. " + + "Lower values make autoscaling react faster.") @Default("10") Integer metricsScrapeIntervalSeconds) { @@ -71,10 +59,8 @@ public record AutoscalingSpec( enabled = enabled != null ? enabled : false; minReplicas = minReplicas != null ? minReplicas : 0; scaleUpThreshold = scaleUpThreshold != null ? scaleUpThreshold : 80; - scaleDownThreshold = scaleDownThreshold != null ? scaleDownThreshold : 20; - cooldownSeconds = cooldownSeconds != null ? cooldownSeconds : 600; scaleUpStabilizationSeconds = scaleUpStabilizationSeconds != null ? scaleUpStabilizationSeconds : 60; - scaleDownStabilizationSeconds = scaleDownStabilizationSeconds != null ? scaleDownStabilizationSeconds : 300; + scaleDownStabilizationSeconds = scaleDownStabilizationSeconds != null ? scaleDownStabilizationSeconds : 600; gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 3600; metricsScrapeIntervalSeconds = metricsScrapeIntervalSeconds != null ? metricsScrapeIntervalSeconds : 10; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index b81ad83b41b7..e4703e8294a7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -52,7 +52,7 @@ public record HiveServer2Spec( ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") ProbeSpec livenessProbe, - @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") AutoscalingSpec autoscaling) { public HiveServer2Spec { @@ -62,6 +62,6 @@ public record HiveServer2Spec( extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 80, 20, null, null, 600, 60, 300, 300, 10); + false, 1, 80, 60, 600, 300, 10); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index c24bac5a1116..79d560104ca7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -56,7 +56,7 @@ public record LlapSpec( String serviceHosts, @JsonPropertyDescription("Readiness probe configuration") ProbeSpec readinessProbe, - @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") AutoscalingSpec autoscaling) { public LlapSpec { @@ -68,7 +68,7 @@ public record LlapSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 1, 0, null, null, 900, 60, 300, 600, 10); + false, 0, 1, 60, 900, 600, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 61c2cf0635e2..8a45f545fb1f 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -57,7 +57,7 @@ public record MetastoreSpec( ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") ProbeSpec livenessProbe, - @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") AutoscalingSpec autoscaling) { public MetastoreSpec { @@ -69,7 +69,7 @@ public record MetastoreSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 75, 30, null, null, 300, 60, 300, 60, 10); + false, 1, 75, 60, 300, 60, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index 716e4025c50d..96e6933030c2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -53,7 +53,7 @@ public record TezAmSpec( @JsonPropertyDescription("StorageClass for the shared scratch PVC. " + "Must support ReadWriteMany access. If null, uses cluster default.") String scratchStorageClassName, - @JsonPropertyDescription("Autoscaling configuration (requires KEDA installed in the cluster)") + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") AutoscalingSpec autoscaling) { public TezAmSpec { @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 5, 10, null, null, 600, 60, 300, 120, 10); + false, 0, 0, 60, 600, 120, 10); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java new file mode 100644 index 000000000000..8d775b9e857a --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java @@ -0,0 +1,89 @@ +/* + * 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.hive.kubernetes.operator.model.status; + +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonInclude; + +/** + * Autoscaling status for a component, surfacing the operator's scaling decisions + * in the HiveCluster status subresource (replaces kubectl get hpa). + */ +@JsonInclude(JsonInclude.Include.NON_NULL) +public class AutoscalingStatus { + + private int currentMetricValue; + private Integer scaleUpThreshold; + private int proposedReplicas; + private String lastScaleTime; + + public int getCurrentMetricValue() { + return currentMetricValue; + } + + public void setCurrentMetricValue(int currentMetricValue) { + this.currentMetricValue = currentMetricValue; + } + + public Integer getScaleUpThreshold() { + return scaleUpThreshold; + } + + public void setScaleUpThreshold(Integer scaleUpThreshold) { + this.scaleUpThreshold = scaleUpThreshold; + } + + public int getProposedReplicas() { + return proposedReplicas; + } + + public void setProposedReplicas(int proposedReplicas) { + this.proposedReplicas = proposedReplicas; + } + + public String getLastScaleTime() { + return lastScaleTime; + } + + public void setLastScaleTime(String lastScaleTime) { + this.lastScaleTime = lastScaleTime; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AutoscalingStatus that = (AutoscalingStatus) o; + return currentMetricValue == that.currentMetricValue + && Objects.equals(scaleUpThreshold, that.scaleUpThreshold) + && proposedReplicas == that.proposedReplicas + && Objects.equals(lastScaleTime, that.lastScaleTime); + } + + @Override + public int hashCode() { + return Objects.hash(currentMetricValue, scaleUpThreshold, + proposedReplicas, lastScaleTime); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/ComponentStatus.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/ComponentStatus.java index 155c46f3a714..2b4f87b37e04 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/ComponentStatus.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/ComponentStatus.java @@ -23,24 +23,43 @@ */ public class ComponentStatus { + private int maxReplicas; + private int minReplicas; + private int currentReplicas; private int readyReplicas; - private int desiredReplicas; private String phase; + private AutoscalingStatus autoscaling; - public int getReadyReplicas() { - return readyReplicas; + public int getMaxReplicas() { + return maxReplicas; } - public void setReadyReplicas(int readyReplicas) { - this.readyReplicas = readyReplicas; + public void setMaxReplicas(int maxReplicas) { + this.maxReplicas = maxReplicas; + } + + public int getMinReplicas() { + return minReplicas; + } + + public void setMinReplicas(int minReplicas) { + this.minReplicas = minReplicas; + } + + public int getCurrentReplicas() { + return currentReplicas; } - public int getDesiredReplicas() { - return desiredReplicas; + public void setCurrentReplicas(int currentReplicas) { + this.currentReplicas = currentReplicas; } - public void setDesiredReplicas(int desiredReplicas) { - this.desiredReplicas = desiredReplicas; + public int getReadyReplicas() { + return readyReplicas; + } + + public void setReadyReplicas(int readyReplicas) { + this.readyReplicas = readyReplicas; } public String getPhase() { @@ -51,6 +70,14 @@ public void setPhase(String phase) { this.phase = phase; } + public AutoscalingStatus getAutoscaling() { + return autoscaling; + } + + public void setAutoscaling(AutoscalingStatus autoscaling) { + this.autoscaling = autoscaling; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -60,12 +87,15 @@ public boolean equals(Object o) { return false; } ComponentStatus that = (ComponentStatus) o; - return readyReplicas == that.readyReplicas && desiredReplicas == that.desiredReplicas && java.util.Objects.equals( - phase, that.phase); + return maxReplicas == that.maxReplicas && minReplicas == that.minReplicas + && currentReplicas == that.currentReplicas && readyReplicas == that.readyReplicas + && java.util.Objects.equals(phase, that.phase) + && java.util.Objects.equals(autoscaling, that.autoscaling); } @Override public int hashCode() { - return java.util.Objects.hash(readyReplicas, desiredReplicas, phase); + return java.util.Objects.hash(maxReplicas, minReplicas, currentReplicas, + readyReplicas, phase, autoscaling); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 71453ef0335b..eb1970480cc5 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -18,25 +18,31 @@ package org.apache.hive.kubernetes.operator.reconciler; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; -import java.util.function.Function; import io.fabric8.kubernetes.api.model.Condition; import io.fabric8.kubernetes.api.model.HasMetadata; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; import io.javaoperatorsdk.operator.api.reconciler.Context; import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration; import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; import io.javaoperatorsdk.operator.api.reconciler.Reconciler; import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; +import org.apache.hive.kubernetes.operator.autoscaling.HiveClusterAutoscaler; +import org.apache.hive.kubernetes.operator.autoscaling.MetricsScraper; import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.HiveClusterStatus; +import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,6 +56,8 @@ public class HiveClusterReconciler implements Reconciler { private static final Logger LOG = LoggerFactory.getLogger(HiveClusterReconciler.class); + private volatile HiveClusterAutoscaler autoscaler; + @Override public UpdateControl reconcile(HiveCluster resource, Context context) { LOG.debug("Reconciling HiveCluster: {}/{} generation={}", @@ -60,7 +68,26 @@ public UpdateControl reconcile(HiveCluster resource, Context entry : eval.patches().entrySet()) { + patchReplicas(client, resource, entry.getKey(), entry.getValue()); + } + // Set autoscaling status on each component + applyAutoscalingStatuses(newStatus, eval.statuses()); + // Always patch status when autoscaling is active (metrics change each scrape) + int interval = getMinScrapeInterval(resource.getSpec()); + resource.setStatus(newStatus); + return UpdateControl.patchStatus(resource) + .rescheduleAfter(Duration.ofSeconds(interval)); + } + + if (!statusChanged) { return UpdateControl.noUpdate(); } @@ -117,20 +144,15 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // Metastore status boolean metastoreReady; if (resource.getSpec().metastore().isEnabled()) { - // When autoscaling, desired = minReplicas (KEDA manages beyond that) - int metastoreDesired = resource.getSpec().metastore().autoscaling().isEnabled() + int msMin = resource.getSpec().metastore().autoscaling().isEnabled() ? Math.max(1, resource.getSpec().metastore().autoscaling().minReplicas()) : resource.getSpec().metastore().replicas(); ComponentStatus metastoreStatus = buildComponentStatus(context, Deployment.class, resource.getMetadata().getName() + "-metastore", - metastoreDesired, - d -> d.getStatus() != null && d.getStatus().getReadyReplicas() != null ? - d.getStatus().getReadyReplicas() : - 0); + resource.getSpec().metastore().replicas(), msMin); status.setMetastore(metastoreStatus); - metastoreReady = metastoreStatus.getReadyReplicas() >= metastoreStatus.getDesiredReplicas() - && metastoreStatus.getDesiredReplicas() > 0; + metastoreReady = metastoreStatus.getReadyReplicas() >= msMin && msMin > 0; conditions.add(buildCondition("MetastoreReady", metastoreReady ? "True" : "False", metastoreReady ? "DeploymentReady" : "DeploymentNotReady", @@ -141,17 +163,16 @@ private HiveClusterStatus buildStatus(HiveCluster resource, existingConditions)); } - // HiveServer2 status — when scale-to-zero, 0/0 is a valid "ready" state (idle) - int hs2Desired = resource.getSpec().hiveServer2().autoscaling().isEnabled() - ? resource.getSpec().hiveServer2().autoscaling().minReplicas() + // HiveServer2 status + int hs2Min = resource.getSpec().hiveServer2().autoscaling().isEnabled() + ? Math.max(1, resource.getSpec().hiveServer2().autoscaling().minReplicas()) : resource.getSpec().hiveServer2().replicas(); ComponentStatus hs2Status = buildComponentStatus(context, Deployment.class, resource.getMetadata().getName() + "-hiveserver2", - hs2Desired, - d -> d.getStatus() != null && d.getStatus().getReadyReplicas() != null ? d.getStatus().getReadyReplicas() : 0); + resource.getSpec().hiveServer2().replicas(), hs2Min); status.setHiveServer2(hs2Status); - boolean hs2Ready = hs2Status.getReadyReplicas() >= hs2Status.getDesiredReplicas(); + boolean hs2Ready = hs2Status.getReadyReplicas() >= hs2Min; conditions.add(buildCondition("HiveServer2Ready", hs2Ready ? "True" : "False", hs2Ready ? "DeploymentReady" : "DeploymentNotReady", hs2Ready ? "HiveServer2 is ready" : "HiveServer2 not yet ready", @@ -159,25 +180,22 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // LLAP status (optional) if (resource.getSpec().llap().isEnabled()) { - int llapDesired = resource.getSpec().llap().autoscaling().isEnabled() + int llapMin = resource.getSpec().llap().autoscaling().isEnabled() ? resource.getSpec().llap().autoscaling().minReplicas() : resource.getSpec().llap().replicas(); status.setLlap(buildComponentStatus(context, StatefulSet.class, resource.getMetadata().getName() + "-llap", - llapDesired, - s -> s.getStatus() != null && s.getStatus().getReadyReplicas() != null ? - s.getStatus().getReadyReplicas() : 0)); + resource.getSpec().llap().replicas(), llapMin)); } // TezAM status (optional) if (resource.getSpec().tezAm().isEnabled()) { - int tezAmDesired = resource.getSpec().tezAm().autoscaling().isEnabled() + int tezAmMin = resource.getSpec().tezAm().autoscaling().isEnabled() ? resource.getSpec().tezAm().autoscaling().minReplicas() : resource.getSpec().tezAm().replicas(); - status.setTezAm(buildComponentStatus(context, StatefulSet.class, resource.getMetadata().getName() + "-tezam", - tezAmDesired, - s -> s.getStatus() != null && - s.getStatus().getReadyReplicas() != null ? s.getStatus().getReadyReplicas() : 0)); + status.setTezAm(buildComponentStatus(context, StatefulSet.class, + resource.getMetadata().getName() + "-tezam", + resource.getSpec().tezAm().replicas(), tezAmMin)); } // Overall Ready condition @@ -197,19 +215,51 @@ private HiveClusterStatus buildStatus(HiveCluster resource, */ private ComponentStatus buildComponentStatus( Context context, Class resourceClass, String expectedResourceName, - int desiredReplicas, Function readyExtractor) { + int maxReplicas, int minReplicas) { ComponentStatus cs = new ComponentStatus(); - cs.setDesiredReplicas(desiredReplicas); + cs.setMaxReplicas(maxReplicas); + cs.setMinReplicas(minReplicas); - int ready = context.getSecondaryResources(resourceClass).stream() + // Read actual spec.replicas and readyReplicas from the live workload + var workload = context.getSecondaryResources(resourceClass).stream() .filter(r -> r.getMetadata().getName().equals(expectedResourceName)) - .findFirst() - .map(readyExtractor) - .orElse(0); + .findFirst(); + + int currentReplicas = workload.map(r -> { + if (r instanceof Deployment d) { + return d.getSpec() != null && d.getSpec().getReplicas() != null + ? d.getSpec().getReplicas() : 0; + } else if (r instanceof StatefulSet s) { + return s.getSpec() != null && s.getSpec().getReplicas() != null + ? s.getSpec().getReplicas() : 0; + } + return 0; + }).orElse(0); + + int ready = workload.map(r -> { + if (r instanceof Deployment d) { + return d.getStatus() != null && d.getStatus().getReadyReplicas() != null + ? d.getStatus().getReadyReplicas() : 0; + } else if (r instanceof StatefulSet s) { + return s.getStatus() != null && s.getStatus().getReadyReplicas() != null + ? s.getStatus().getReadyReplicas() : 0; + } + return 0; + }).orElse(0); + cs.setCurrentReplicas(currentReplicas); cs.setReadyReplicas(ready); - cs.setPhase(ready >= desiredReplicas && desiredReplicas > 0 ? "Running" : "Pending"); + + if (currentReplicas == 0 && ready == 0) { + cs.setPhase("Idle"); + } else if (ready >= currentReplicas && currentReplicas > 0) { + cs.setPhase("Running"); + } else if (currentReplicas == 0 && ready > 0) { + cs.setPhase("ScalingDown"); + } else { + cs.setPhase("Pending"); + } return cs; } @@ -304,4 +354,79 @@ private boolean conditionsEqualIgnoringTime(List a, List b } return true; } + + private void applyAutoscalingStatuses(HiveClusterStatus status, + Map statuses) { + if (statuses.containsKey("hiveserver2") && status.getHiveServer2() != null) { + status.getHiveServer2().setAutoscaling(statuses.get("hiveserver2")); + } + if (statuses.containsKey("metastore") && status.getMetastore() != null) { + status.getMetastore().setAutoscaling(statuses.get("metastore")); + } + if (statuses.containsKey("llap") && status.getLlap() != null) { + status.getLlap().setAutoscaling(statuses.get("llap")); + } + if (statuses.containsKey("tezam") && status.getTezAm() != null) { + status.getTezAm().setAutoscaling(statuses.get("tezam")); + } + } + + // --- Autoscaling helpers --- + + private HiveClusterAutoscaler getOrCreateAutoscaler(KubernetesClient client) { + if (autoscaler == null) { + synchronized (this) { + if (autoscaler == null) { + autoscaler = new HiveClusterAutoscaler(new MetricsScraper(client)); + } + } + } + return autoscaler; + } + + private static boolean anyAutoscalingEnabled(HiveClusterSpec spec) { + if (spec.hiveServer2().autoscaling().isEnabled()) { + return true; + } + if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { + return true; + } + if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { + return true; + } + if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { + return true; + } + return false; + } + + private static int getMinScrapeInterval(HiveClusterSpec spec) { + int min = Integer.MAX_VALUE; + if (spec.hiveServer2().autoscaling().isEnabled()) { + min = Math.min(min, spec.hiveServer2().autoscaling().metricsScrapeIntervalSeconds()); + } + if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { + min = Math.min(min, spec.metastore().autoscaling().metricsScrapeIntervalSeconds()); + } + if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { + min = Math.min(min, spec.llap().autoscaling().metricsScrapeIntervalSeconds()); + } + if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { + min = Math.min(min, spec.tezAm().autoscaling().metricsScrapeIntervalSeconds()); + } + return min == Integer.MAX_VALUE ? 10 : min; + } + + private void patchReplicas(KubernetesClient client, HiveCluster resource, + String component, int replicas) { + String namespace = resource.getMetadata().getNamespace(); + String workloadName = resource.getMetadata().getName() + "-" + component; + if ("llap".equals(component) || "tezam".equals(component)) { + client.apps().statefulSets().inNamespace(namespace).withName(workloadName).scale(replicas); + LOG.info("Scaled StatefulSet {}/{} to {} replicas", namespace, workloadName, replicas); + } else { + client.apps().deployments().inNamespace(namespace).withName(workloadName).scale(replicas); + LOG.info("Scaled Deployment {}/{} to {} replicas", namespace, workloadName, replicas); + } + } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java index 46aa53890573..111a496672da 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java @@ -29,9 +29,7 @@ import io.javaoperatorsdk.operator.processing.dependent.workflow.Condition; import org.apache.hive.kubernetes.operator.dependent.HiveConfigMapDependent; import org.apache.hive.kubernetes.operator.dependent.HiveServer2DeploymentDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveServer2InterceptorRouteDependent; import org.apache.hive.kubernetes.operator.dependent.HivePdbDependent; -import org.apache.hive.kubernetes.operator.dependent.HiveScaledObjectDependent; import org.apache.hive.kubernetes.operator.dependent.HiveServiceDependent; import org.apache.hive.kubernetes.operator.dependent.LlapStatefulSetDependent; import org.apache.hive.kubernetes.operator.dependent.MetastoreDeploymentDependent; @@ -72,9 +70,6 @@ public final class HiveWorkflowSpec implements WorkflowSpec { private static final Condition HS2_AUTOSCALING = (dr, primary, ctx) -> primary.getSpec().hiveServer2().autoscaling().isEnabled(); - private static final Condition HS2_SCALE_TO_ZERO = - (dr, primary, ctx) -> primary.getSpec().hiveServer2().autoscaling().isEnabled() - && primary.getSpec().hiveServer2().autoscaling().minReplicas() == 0; // SPECS must be declared AFTER all conditions to avoid static init order issues. private static final List SPECS = buildSpecs(); @@ -152,30 +147,6 @@ private static List buildSpecs() { Set.of("hiveserver2-configmap", "hadoop-configmap", "tezam-service", "scratch-pvc"), null, null, null, TEZAM_ENABLED, null)); - specs.add(new DependentResourceSpec( - HiveScaledObjectDependent.HiveServer2.class, "hs2-scaledobject", - Set.of("hiveserver2-deployment"), - null, HS2_AUTOSCALING, null, null, null)); - - specs.add(new DependentResourceSpec( - HiveServer2InterceptorRouteDependent.class, "hs2-interceptor-route", - Set.of("hiveserver2-deployment"), - null, HS2_SCALE_TO_ZERO, null, null, null)); - - specs.add(new DependentResourceSpec( - HiveScaledObjectDependent.Metastore.class, "metastore-scaledobject", - Set.of("metastore-deployment"), - null, METASTORE_AUTOSCALING, null, null, null)); - - specs.add(new DependentResourceSpec( - HiveScaledObjectDependent.Llap.class, "llap-scaledobject", - Set.of("llap-statefulset", "hs2-scaledobject"), - null, LLAP_AUTOSCALING, null, null, null)); - - specs.add(new DependentResourceSpec( - HiveScaledObjectDependent.TezAm.class, "tezam-scaledobject", - Set.of("tezam-statefulset", "hs2-scaledobject"), - null, TEZAM_AUTOSCALING, null, null, null)); // --- Autoscaling: PodDisruptionBudgets (conditional) --- specs.add(new DependentResourceSpec( diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 95e66bd91979..0415865f1c9d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -94,6 +94,10 @@ private ConfigUtils() { public static final String HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY = "hive.server2.tez.sessions.per.default.queue"; public static final int HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT = 1; + public static final String HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_KEY = + "hive.server2.tez.initialize.default.sessions"; + public static final boolean HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_DEFAULT = true; + public static final String HIVE_SERVER2_THRIFT_PORT_KEY = "hive.server2.thrift.port"; public static final int HIVE_SERVER2_THRIFT_PORT_DEFAULT = 10000; @@ -150,4 +154,15 @@ public static int getInt(Map overrides, } return defaultVal; } + + public static boolean getBoolean(Map overrides, + String key, boolean defaultVal) { + if (overrides != null) { + String val = overrides.get(key); + if (val != null) { + return Boolean.parseBoolean(val); + } + } + return defaultVal; + } } From 5d0b6564f5db6773f7c055e542a467099f4ff4aa Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Sun, 31 May 2026 19:21:37 +0530 Subject: [PATCH 08/22] Add HS2 Priority for Auto Scaling --- .../hive-operator/templates/clusterrole.yaml | 4 +- .../autoscaling/HiveClusterAutoscaler.java | 44 +++++- .../autoscaling/PrometheusTextParser.java | 129 ++++++++++-------- .../reconciler/HiveClusterReconciler.java | 6 +- 4 files changed, 113 insertions(+), 70 deletions(-) diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml index ba189be3dd2b..3b0eb0e8e40f 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/clusterrole.yaml @@ -50,10 +50,10 @@ rules: - apiGroups: [""] resources: ["events"] verbs: ["create", "patch"] - # Pods: read-only for readiness checking + # Pods: read + patch (patch needed for pod-deletion-cost annotation) - apiGroups: [""] resources: ["pods"] - verbs: ["get", "list", "watch"] + verbs: ["get", "list", "watch", "patch"] # PodDisruptionBudgets for graceful autoscaling - apiGroups: ["policy"] resources: ["poddisruptionbudgets"] diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 5abfc710f7b8..7a70a48c9e72 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -89,9 +89,12 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie // HiveServer2 if (spec.hiveServer2().autoscaling().isEnabled()) { + Map hs2Selector = Labels.selectorForComponent(cluster, "hiveserver2"); + List hs2Metrics = scraper.scrape(namespace, hs2Selector); + updatePodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); evaluateComponent(cluster, client, namespace, clusterName, "hiveserver2", spec.hiveServer2().autoscaling(), - spec.hiveServer2().replicas(), patches, statuses); + spec.hiveServer2().replicas(), patches, statuses, hs2Metrics); } // Metastore @@ -131,13 +134,27 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, String namespace, String clusterName, String component, AutoscalingSpec autoscaling, int maxReplicas, Map patches, Map statuses) { + evaluateComponent(cluster, client, namespace, clusterName, component, + autoscaling, maxReplicas, patches, statuses, null); + } + + private void evaluateComponent(HiveCluster cluster, KubernetesClient client, + String namespace, String clusterName, String component, + AutoscalingSpec autoscaling, int maxReplicas, + Map patches, Map statuses, + List preScrapedMetrics) { int currentReplicas = getCurrentReplicas(client, namespace, clusterName, component); String key = namespace + "/" + clusterName + "/" + component; - Map selector = Labels.selectorForComponent(cluster, component); - List metrics = scraper.scrape(namespace, selector); + List metrics; + if (preScrapedMetrics != null) { + metrics = preScrapedMetrics; + } else { + Map selector = Labels.selectorForComponent(cluster, component); + metrics = scraper.scrape(namespace, selector); + } // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), // not their own pod metrics. Allow evaluation even with 0 own pods. @@ -204,4 +221,25 @@ private int getCurrentReplicas(KubernetesClient client, String namespace, } } + /** + * Patches each pod's deletion cost annotation based on its active session count. + * Kubernetes uses this during scale-down to kill idle pods first (lower cost = killed first). + */ + private void updatePodDeletionCost(KubernetesClient client, String namespace, + List metrics, String metricName) { + for (PodMetrics pm : metrics) { + int sessions = pm.metrics().getOrDefault(metricName, 0.0).intValue(); + try { + client.pods().inNamespace(namespace).withName(pm.podName()) + .edit(pod -> { + pod.getMetadata().getAnnotations() + .put("controller.kubernetes.io/pod-deletion-cost", String.valueOf(sessions)); + return pod; + }); + } catch (Exception e) { + LOG.debug("Failed to update deletion cost for pod {}: {}", pm.podName(), e.getMessage()); + } + } + } + } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java index 88f6dce6fac8..3f8ab6527703 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -18,6 +18,9 @@ package org.apache.hive.kubernetes.operator.autoscaling; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; import java.util.HashMap; import java.util.Map; @@ -41,42 +44,43 @@ public static Map parse(String body) { if (body == null || body.isEmpty()) { return result; } - for (String line : body.split("\n")) { - if (line.isEmpty() || line.charAt(0) == '#') { - continue; - } - // Format: metric_name[{labels}] value [timestamp] - // We extract metric_name (without labels) and value. - String metricKey; - String valuePart; - int braceStart = line.indexOf('{'); - if (braceStart >= 0) { - int braceEnd = line.indexOf('}', braceStart); - if (braceEnd < 0) { + try (BufferedReader reader = new BufferedReader(new StringReader(body))) { + String line; + while ((line = reader.readLine()) != null) { + if (line.isEmpty() || line.charAt(0) == '#') { continue; } - metricKey = line.substring(0, braceStart); - valuePart = line.substring(braceEnd + 1).trim(); - } else { - int spaceIdx = line.indexOf(' '); - if (spaceIdx < 0) { - continue; + String metricKey; + String valuePart; + int braceStart = line.indexOf('{'); + if (braceStart >= 0) { + int braceEnd = line.indexOf('}', braceStart); + if (braceEnd < 0) { + continue; + } + metricKey = line.substring(0, braceStart); + valuePart = line.substring(braceEnd + 1).trim(); + } else { + int spaceIdx = line.indexOf(' '); + if (spaceIdx < 0) { + continue; + } + metricKey = line.substring(0, spaceIdx); + valuePart = line.substring(spaceIdx + 1).trim(); + } + int spaceInValue = valuePart.indexOf(' '); + if (spaceInValue > 0) { + valuePart = valuePart.substring(0, spaceInValue); + } + try { + double value = Double.parseDouble(valuePart); + result.merge(metricKey, value, Double::sum); + } catch (NumberFormatException e) { + // Skip NaN, +Inf, -Inf, or malformed values } - metricKey = line.substring(0, spaceIdx); - valuePart = line.substring(spaceIdx + 1).trim(); - } - // valuePart may contain "value timestamp" — take only value - int spaceInValue = valuePart.indexOf(' '); - if (spaceInValue > 0) { - valuePart = valuePart.substring(0, spaceInValue); - } - try { - double value = Double.parseDouble(valuePart); - // Sum duplicates (multiple label sets for same metric name) - result.merge(metricKey, value, Double::sum); - } catch (NumberFormatException e) { - // Skip NaN, +Inf, -Inf, or malformed values } + } catch (IOException e) { + // StringReader does not throw IOException } return result; } @@ -90,38 +94,43 @@ public static Map parseWithLabels(String body) { if (body == null || body.isEmpty()) { return result; } - for (String line : body.split("\n")) { - if (line.isEmpty() || line.charAt(0) == '#') { - continue; - } - String metricKey; - String valuePart; - int braceStart = line.indexOf('{'); - if (braceStart >= 0) { - int braceEnd = line.indexOf('}', braceStart); - if (braceEnd < 0) { + try (BufferedReader reader = new BufferedReader(new StringReader(body))) { + String line; + while ((line = reader.readLine()) != null) { + if (line.isEmpty() || line.charAt(0) == '#') { continue; } - metricKey = line.substring(0, braceEnd + 1); - valuePart = line.substring(braceEnd + 1).trim(); - } else { - int spaceIdx = line.indexOf(' '); - if (spaceIdx < 0) { - continue; + String metricKey; + String valuePart; + int braceStart = line.indexOf('{'); + if (braceStart >= 0) { + int braceEnd = line.indexOf('}', braceStart); + if (braceEnd < 0) { + continue; + } + metricKey = line.substring(0, braceEnd + 1); + valuePart = line.substring(braceEnd + 1).trim(); + } else { + int spaceIdx = line.indexOf(' '); + if (spaceIdx < 0) { + continue; + } + metricKey = line.substring(0, spaceIdx); + valuePart = line.substring(spaceIdx + 1).trim(); + } + int spaceInValue = valuePart.indexOf(' '); + if (spaceInValue > 0) { + valuePart = valuePart.substring(0, spaceInValue); + } + try { + double value = Double.parseDouble(valuePart); + result.put(metricKey, value); + } catch (NumberFormatException e) { + // Skip } - metricKey = line.substring(0, spaceIdx); - valuePart = line.substring(spaceIdx + 1).trim(); - } - int spaceInValue = valuePart.indexOf(' '); - if (spaceInValue > 0) { - valuePart = valuePart.substring(0, spaceInValue); - } - try { - double value = Double.parseDouble(valuePart); - result.put(metricKey, value); - } catch (NumberFormatException e) { - // Skip } + } catch (IOException e) { + // StringReader does not throw IOException } return result; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index eb1970480cc5..96110762dd1d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -375,11 +375,7 @@ private void applyAutoscalingStatuses(HiveClusterStatus status, private HiveClusterAutoscaler getOrCreateAutoscaler(KubernetesClient client) { if (autoscaler == null) { - synchronized (this) { - if (autoscaler == null) { - autoscaler = new HiveClusterAutoscaler(new MetricsScraper(client)); - } - } + autoscaler = new HiveClusterAutoscaler(new MetricsScraper(client)); } return autoscaler; } From a4af13a5b843b9fa28ab0105d6563c4fb266900f Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 1 Jun 2026 14:43:12 +0530 Subject: [PATCH 09/22] CleanUp --- .../hive/registry/impl/TezAmInstance.java | 1 + packaging/src/kubernetes/README.md | 4 ++-- .../kubernetes/helm/hive-operator/values.yaml | 14 ++++++------- .../dependent/TezAmStatefulSetDependent.java | 21 ------------------- 4 files changed, 10 insertions(+), 30 deletions(-) diff --git a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java index b31175e0f1f6..e6ad084f3a40 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java +++ b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java @@ -1,3 +1,4 @@ + /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index 54e58393fb85..89869c919860 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -540,7 +540,7 @@ When `autoscaling.enabled: true` is set for a component, the operator: - HS2: deregisters from ZK, drains open sessions, kills JVM - HMS: kills JVM (stateless HTTP — no drain needed) - LLAP: waits until all executors become idle, kills JVM - - TezAM: waits for current DAG completion, kills JVM + - TezAM: no drain (DAGAppMaster does not expose JMX metrics) 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety cap) 6. Pod terminates immediately once drain completes (does NOT wait the full grace period — it's only the upper safety bound) @@ -957,7 +957,7 @@ setup is needed — simply connect to HS2 and the operator wakes LLAP/TezAM as n |-------|---------|-------------| | `cluster..autoscaling.enabled` | `false` | Enable operator-driven autoscaling for this component | | `cluster..autoscaling.minReplicas` | `0` | Floor replica count. 0 enables scale-to-zero (LLAP, TezAM only; HS2 minimum is 1) | -| `cluster..autoscaling.scaleUpThreshold` | `80` | Metric threshold triggering scale-up (total sessions for HS2, request rate for HMS, busy slots for LLAP, demand per HS2 pod for TezAM) | +| `cluster..autoscaling.scaleUpThreshold` | `100` (HS2/HMS), `10` (LLAP) | Metric threshold per pod triggering scale-up (sessions for HS2, connections for HMS, busy slots for LLAP). TezAM scales 1:1 with demand (no threshold). | | `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | Stabilization window for scale-up decisions (prevents flapping) | | `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down decisions (also acts as cooldown between consecutive scale-downs) | | `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap (seconds) — pod terminates immediately once drain completes, this is only the upper bound | diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index f122686a9cad..f6b3cbbd3542 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -116,9 +116,9 @@ cluster: # The operator scrapes JMX Exporter metrics from pods directly. # When enabled, 'replicas' above acts as the max replica ceiling. autoscaling: - enabled: true + enabled: false minReplicas: 1 - scaleUpThreshold: 2 + scaleUpThreshold: 100 scaleUpStabilizationSeconds: 60 scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 60 @@ -141,9 +141,9 @@ cluster: # Autoscaling (operator-driven, no external dependencies) # When enabled, 'replicas' above acts as the max replica ceiling autoscaling: - enabled: true + enabled: false minReplicas: 1 - scaleUpThreshold: 1 + scaleUpThreshold: 100 scaleUpStabilizationSeconds: 60 scaleDownStabilizationSeconds: 600 gracePeriodSeconds: 300 @@ -166,9 +166,9 @@ cluster: # minReplicas: 0 enables scale-to-zero — scales up when HS2 has active sessions # When enabled, 'replicas' above acts as the max replica ceiling autoscaling: - enabled: true + enabled: false minReplicas: 0 - scaleUpThreshold: 1 + scaleUpThreshold: 10 scaleUpStabilizationSeconds: 60 scaleDownStabilizationSeconds: 900 gracePeriodSeconds: 600 @@ -192,7 +192,7 @@ cluster: # TezAM scales demand-based: max(totalSessions, hs2Pods * sessionsPerQueue) # No scaleUpThreshold needed — scaling is 1:1 with session demand autoscaling: - enabled: true + enabled: false minReplicas: 0 scaleUpStabilizationSeconds: 60 scaleDownStabilizationSeconds: 600 diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index 3b14cf54b494..c2c3352473b1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -127,13 +127,6 @@ protected StatefulSet desired(HiveCluster hiveCluster, replaceConfMountWithSubPaths(volumeMounts, "hive-config", "hive-site.xml", "tez-site.xml", "core-site.xml"); - // Add Prometheus JMX Exporter when autoscaling is enabled - AutoscalingSpec autoscaling = tezAm.autoscaling(); - if (autoscaling.isEnabled()) { - addJmxExporter(spec.image(), COMPONENT, - initContainers, volumeMounts, volumes, envVars, ports); - } - // Pre-compute config hash for the pod template annotation. // TezAM uses the same ConfigMaps as HS2 (hive-site.xml + tez-site.xml + core-site.xml). String configHash = sha256( @@ -185,20 +178,6 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); - // Graceful scale-down: poll JMX Exporter (port 9404) for DAGsRunning to reach 0. - if (autoscaling.isEnabled()) { - String preStopScript = buildDrainScript( - "Waiting for active DAGs to complete", - "tez_am_dagsrunning", "DAGS", - "No active DAGs. Safe to terminate Tez AM.", - 10, 6, null); - applyAutoscalingLifecycle( - statefulSet.getSpec().getTemplate().getSpec(), - statefulSet.getSpec().getTemplate().getMetadata(), - preStopScript, autoscaling.gracePeriodSeconds(), - autoscaling.metricsScrapeIntervalSeconds()); - } - appendUserVolumes(statefulSet.getSpec().getTemplate().getSpec(), spec.volumes(), spec.volumeMounts(), tezAm.extraVolumes(), tezAm.extraVolumeMounts()); From 50000c0c2175b2135a6db5f656a8964fa6c2b36d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 2 Jun 2026 15:53:22 +0530 Subject: [PATCH 10/22] Add CPU for HS2 & HMS --- .../hive/registry/impl/TezAmInstance.java | 1 - packaging/src/kubernetes/README.md | 105 +++++++++++++++++- .../crds/hiveclusters.hive.apache.org-v1.yml | 68 ++++++++++++ .../hive-operator/templates/hivecluster.yaml | 4 + .../kubernetes/helm/hive-operator/values.yaml | 4 + packaging/src/kubernetes/pom.xml | 1 + .../autoscaling/ComponentAutoscaler.java | 57 +++++++++- .../autoscaling/HiveClusterAutoscaler.java | 7 ++ .../dependent/HiveDependentResource.java | 8 ++ .../operator/model/spec/AutoscalingSpec.java | 12 +- .../operator/model/spec/HiveServer2Spec.java | 2 +- .../operator/model/spec/LlapSpec.java | 2 +- .../operator/model/spec/MetastoreSpec.java | 2 +- .../operator/model/spec/TezAmSpec.java | 2 +- .../model/status/AutoscalingStatus.java | 31 ++++++ 15 files changed, 292 insertions(+), 14 deletions(-) diff --git a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java index e6ad084f3a40..b31175e0f1f6 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java +++ b/llap-client/src/java/org/apache/hadoop/hive/registry/impl/TezAmInstance.java @@ -1,4 +1,3 @@ - /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index 89869c919860..6d18038078a3 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -571,8 +571,8 @@ The autoscaling system uses three independent timing controls: | Component | Scale-Up Formula | Scale-Down | JMX Metric | |-----------|-----------------|------------|------------| -| **HiveServer2** | `ceil(sum(hs2_open_sessions) / scaleUpThreshold)` | Sessions drop to 0 → scale to minReplicas | `hs2_open_sessions` | -| **Metastore** | `ceil(api_request_rate / scaleUpThreshold)` | Rate drops to 0 → scale to minReplicas | `api_*_total` (operator computes delta/time) | +| **HiveServer2** | `max(ceil(sessions / threshold), cpu_desired)` | Sessions drop to 0 AND CPU below threshold → scale to minReplicas | `hs2_open_sessions`, `jvm_process_cpu_load` | +| **Metastore** | `max(ceil(api_rate / threshold), cpu_desired)` | Rate drops to 0 AND CPU below threshold → scale to minReplicas | `api_*_total`, `jvm_process_cpu_load` | | **LLAP** | `ceil(avg(queued + configured - available) / scaleUpThreshold)` | All executors idle + no HS2 sessions | `hadoop_llapdaemon_executor*` | | **Tez AM** | `max(sum(hs2_open_sessions), count(HS2_pods) * sessions_per_queue)` | All HS2 sessions closed | `hs2_open_sessions` (from HS2 pods) | @@ -637,6 +637,99 @@ terminates immediately once sessions drain, not after the full grace period. | **LLAP** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | | **TezAM** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | +### CPU-Based Scaling (HS2 and HMS) + +In addition to the primary metrics (sessions for HS2, API request rate for HMS), +the operator supports a secondary **CPU-based scaling signal** for HiveServer2 and +Metastore. The final desired replica count is: + +``` +final_desired = max(metric_desired, cpu_desired) +``` + +Either signal can trigger scale-up; neither can force scale-down below what the +other recommends. CPU-based scaling uses the same stabilization windows as metric-based +scaling (no separate CPU stabilization). + +**How it works:** + +1. The operator scrapes `ProcessCpuLoad` from `java.lang:type=OperatingSystem` via JMX + Exporter (exported as `jvm_process_cpu_load`, a 0.0–1.0 fraction) +2. Averages across all pods, converts to percentage (0–100) +3. If avg CPU >= `cpuScaleUpThreshold`: scales up proportionally + (`ceil(avgCpu * currentReplicas / cpuScaleUpThreshold)`) +4. If avg CPU < `cpuScaleDownThreshold`: scales down + (`ceil(avgCpu * currentReplicas / cpuScaleUpThreshold)`, floored at `minReplicas`) +5. Between thresholds: holds current replica count + +**Configuration:** + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) that triggers scale-up. Set to `0` to disable CPU-based scaling. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. | + +**Example:** + +```yaml +cluster: + hiveServer2: + replicas: 10 + resources: + limitsCpu: "2" # Recommended: set CPU limits so ProcessCpuLoad is relative to pod allocation + autoscaling: + enabled: true + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 + + metastore: + replicas: 6 + resources: + limitsCpu: "2" + autoscaling: + enabled: true + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 +``` + +**Important: CPU limits and metric accuracy** + +`ProcessCpuLoad` reports CPU usage as a fraction of **available processors**. Without +CPU limits, the JVM sees all node cores (e.g., 8 cores), so even heavy single-pod +load only shows ~12.5%. With `limitsCpu: "2"`, the JVM sees 2 processors and the +metric becomes "% of allocated CPU" — making thresholds meaningful. + +| Pod CPU Limit | JVM sees | 90% threshold means | +|---------------|----------|---------------------| +| None (no limit) | All node cores (e.g., 8) | Using 7.2 of 8 cores — very hard to reach | +| `2` | 2 cores | Using 1.8 of 2 allocated cores | +| `4` | 4 cores | Using 3.6 of 4 allocated cores | + +**Recommendation:** Always set `resources.limitsCpu` when using CPU-based autoscaling. + +**Status output:** + +The operator reports CPU metrics in the HiveCluster status: + +```yaml +status: + hiveServer2: + autoscaling: + currentMetricValue: 5 # total sessions + scaleUpThreshold: 100 + currentCpuPercent: 72.45 # avg ProcessCpuLoad * 100 + cpuScaleUpThreshold: 90 + cpuProposedReplicas: 2 # what CPU alone would recommend + proposedReplicas: 2 + lastScaleTime: "2026-05-31T04:23:07Z" +``` + +**Applicability:** CPU-based scaling only applies to HS2 and HMS. LLAP and TezAM +do not use CPU as a scaling signal (LLAP scales on busy executor slots which already +correlates with CPU; TezAM is demand-based from HS2 session count). + +--- + ### Enabling Autoscaling **CLI (with Ozone storage backend):** @@ -761,8 +854,8 @@ When autoscaling is enabled, the operator automatically: | Component | Key Metrics | Purpose | |-----------|---------|---------| -| **HiveServer2** | `hs2_open_sessions` | Session count — used for HS2 scaling and as activation gate for LLAP/TezAM | -| **Metastore** | `api_*_total` | API call counters (operator computes request rate from deltas) | +| **HiveServer2** | `hs2_open_sessions`, `jvm_process_cpu_load` | Session count for primary scaling + CPU for secondary scaling signal | +| **Metastore** | `api_*_total`, `jvm_process_cpu_load` | API call counters (operator computes request rate from deltas) + CPU for secondary scaling signal | | **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available | | **Tez AM** | N/A (scales on HS2 metrics) | TezAM scaling is demand-driven from `hs2_open_sessions` — no TezAM-specific metrics needed | @@ -803,6 +896,8 @@ cluster: | `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down (picks most conservative recommendation in window). Also acts as cooldown between consecutive scale-downs. | | `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap: max drain time before forced termination. Pod exits immediately once drain completes. | | `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | How often the operator scrapes JMX metrics from pods. Lower = faster reaction. | +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) triggering scale-up. Only HS2/HMS. Set to 0 to disable. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. Only HS2/HMS. | --- @@ -962,6 +1057,8 @@ setup is needed — simply connect to HS2 and the operator wakes LLAP/TezAM as n | `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down decisions (also acts as cooldown between consecutive scale-downs) | | `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap (seconds) — pod terminates immediately once drain completes, this is only the upper bound | | `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | How often the operator polls JMX metrics from pods. Lower = faster reaction time. | +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) triggering scale-up. Only HS2/HMS. Set to 0 to disable. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. Only HS2/HMS. | --- diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 1faee974870c..47f49ddc79d3 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -48,6 +48,17 @@ spec: description: "Autoscaling configuration (operator-driven, no external\ \ dependencies)" properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -203,6 +214,17 @@ spec: description: "Autoscaling configuration (operator-driven, no external\ \ dependencies)" properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -333,6 +355,17 @@ spec: description: "Autoscaling configuration (operator-driven, no external\ \ dependencies)" properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -516,6 +549,17 @@ spec: description: "Autoscaling configuration (operator-driven, no external\ \ dependencies)" properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer enabled: default: false description: Whether autoscaling is enabled for this component @@ -666,6 +710,12 @@ spec: properties: autoscaling: properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number currentMetricValue: type: integer lastScaleTime: @@ -690,6 +740,12 @@ spec: properties: autoscaling: properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number currentMetricValue: type: integer lastScaleTime: @@ -714,6 +770,12 @@ spec: properties: autoscaling: properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number currentMetricValue: type: integer lastScaleTime: @@ -740,6 +802,12 @@ spec: properties: autoscaling: properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number currentMetricValue: type: integer lastScaleTime: diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index f6178ab73a72..1bfea9c3abbc 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -76,6 +76,8 @@ spec: scaleDownStabilizationSeconds: {{ .Values.cluster.metastore.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.metastore.autoscaling.gracePeriodSeconds }} metricsScrapeIntervalSeconds: {{ .Values.cluster.metastore.autoscaling.metricsScrapeIntervalSeconds | default 10 }} + cpuScaleUpThreshold: {{ .Values.cluster.metastore.autoscaling.cpuScaleUpThreshold | default 90 }} + cpuScaleDownThreshold: {{ .Values.cluster.metastore.autoscaling.cpuScaleDownThreshold | default 30 }} {{- end }} {{- else }} {{- if .Values.cluster.metastore.externalUri }} @@ -115,6 +117,8 @@ spec: scaleDownStabilizationSeconds: {{ .Values.cluster.hiveServer2.autoscaling.scaleDownStabilizationSeconds }} gracePeriodSeconds: {{ .Values.cluster.hiveServer2.autoscaling.gracePeriodSeconds }} metricsScrapeIntervalSeconds: {{ .Values.cluster.hiveServer2.autoscaling.metricsScrapeIntervalSeconds | default 10 }} + cpuScaleUpThreshold: {{ .Values.cluster.hiveServer2.autoscaling.cpuScaleUpThreshold | default 90 }} + cpuScaleDownThreshold: {{ .Values.cluster.hiveServer2.autoscaling.cpuScaleDownThreshold | default 30 }} {{- end }} llap: diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index f6b3cbbd3542..88f720d65c7e 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -123,6 +123,8 @@ cluster: scaleDownStabilizationSeconds: 300 gracePeriodSeconds: 60 metricsScrapeIntervalSeconds: 10 + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 # Set to use an external Metastore instead of deploying one: # enabled: false # externalUri: "thrift://external-metastore:9083" @@ -148,6 +150,8 @@ cluster: scaleDownStabilizationSeconds: 600 gracePeriodSeconds: 300 metricsScrapeIntervalSeconds: 10 + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 # --------------------------------------------------------------------------- # LLAP — enabled by default for full-HA diff --git a/packaging/src/kubernetes/pom.xml b/packaging/src/kubernetes/pom.xml index 342441730cc5..7fe9937a5eb7 100644 --- a/packaging/src/kubernetes/pom.xml +++ b/packaging/src/kubernetes/pom.xml @@ -189,6 +189,7 @@ docker build + --no-cache -t apache/hive:operator-${project.version} . diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java index ca435d5f92b2..d8aa95d32dc3 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java @@ -32,22 +32,28 @@ public class ComponentAutoscaler { /** Result of an autoscaling evaluation. */ - public record EvaluationResult(int rawMetricValue, int proposedReplicas, Integer patchTo) {} + public record EvaluationResult(int rawMetricValue, double cpuPercent, + int cpuProposedReplicas, int proposedReplicas, Integer patchTo) {} private static final Logger LOG = LoggerFactory.getLogger(ComponentAutoscaler.class); + private static final String METRIC_CPU_LOAD = "jvm_process_cpu_load"; + private final String component; private final ScalingStrategy strategy; + private final boolean cpuScalingApplicable; private StabilizationWindow scaleUpWindow; private StabilizationWindow scaleDownWindow; private int lastScaleUpStabilization = -1; private int lastScaleDownStabilization = -1; private boolean initialized; + private double lastCpuPercent; public ComponentAutoscaler(String component, ScalingStrategy strategy) { this.component = component; this.strategy = strategy; + this.cpuScalingApplicable = "hiveserver2".equals(component) || "metastore".equals(component); } /** Whether the underlying strategy uses scaleUpThreshold for scaling decisions. */ @@ -74,7 +80,11 @@ public EvaluationResult evaluate(List metrics, AutoscalingSpec spec, int rawDesired = strategy.computeDesiredReplicas(metrics, spec, maxReplicas); int metricValue = strategy.lastMetricValue(); - int clamped = Math.max(spec.minReplicas(), Math.min(rawDesired, maxReplicas)); + + // CPU-based scaling: combine with metric-based desired via max() + int cpuDesired = computeCpuDesired(metrics, spec, currentReplicas); + int combined = Math.max(rawDesired, cpuDesired); + int clamped = Math.max(spec.minReplicas(), Math.min(combined, maxReplicas)); scaleUpWindow.record(clamped); scaleDownWindow.record(clamped); @@ -96,7 +106,7 @@ public EvaluationResult evaluate(List metrics, AutoscalingSpec spec, target = Math.max(spec.minReplicas(), Math.min(target, maxReplicas)); if (target == currentReplicas) { - return new EvaluationResult(metricValue, clamped, null); + return new EvaluationResult(metricValue, lastCpuPercent, cpuDesired, clamped, null); } if (target < currentReplicas) { @@ -104,7 +114,46 @@ public EvaluationResult evaluate(List metrics, AutoscalingSpec spec, } else { LOG.info("[{}] Scaling up: {} -> {}", component, currentReplicas, target); } - return new EvaluationResult(metricValue, clamped, target); + return new EvaluationResult(metricValue, lastCpuPercent, cpuDesired, clamped, target); + } + + /** + * Compute desired replicas based on CPU utilization. + * Returns 0 if CPU scaling is not applicable or no CPU data is available. + */ + private int computeCpuDesired(List metrics, AutoscalingSpec spec, int currentReplicas) { + if (!cpuScalingApplicable || spec.cpuScaleUpThreshold() <= 0 || metrics.isEmpty()) { + lastCpuPercent = 0; + return 0; + } + + double totalCpu = 0; + int count = 0; + for (PodMetrics pm : metrics) { + Double cpu = pm.metrics().get(METRIC_CPU_LOAD); + if (cpu != null) { + totalCpu += cpu * 100.0; + count++; + } + } + if (count == 0) { + lastCpuPercent = 0; + return 0; + } + double avgCpuPercent = totalCpu / count; + lastCpuPercent = avgCpuPercent; + LOG.debug("[{}] CPU raw: totalCpu={}, count={}, avg={}%", component, totalCpu, count, avgCpuPercent); + + if (avgCpuPercent >= spec.cpuScaleUpThreshold()) { + // Scale up proportionally: how many pods to bring avg below threshold + return (int) Math.ceil(avgCpuPercent * currentReplicas / spec.cpuScaleUpThreshold()); + } else if (avgCpuPercent < spec.cpuScaleDownThreshold()) { + // Scale down: current load could fit in fewer pods + int desired = (int) Math.ceil(avgCpuPercent * currentReplicas / spec.cpuScaleUpThreshold()); + return Math.max(desired, spec.minReplicas()); + } + // Between thresholds: hold current + return currentReplicas; } private void ensureWindows(AutoscalingSpec spec) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 7a70a48c9e72..d9989af8e5eb 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -182,6 +182,13 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, if (autoscaler.usesScaleUpThreshold()) { as.setScaleUpThreshold(autoscaling.scaleUpThreshold()); } + // CPU metrics (only for HS2 and HMS — LLAP/TezAM don't use CPU-based scaling) + if (("hiveserver2".equals(component) || "metastore".equals(component)) + && autoscaling.cpuScaleUpThreshold() > 0) { + as.setCurrentCpuPercent(result.cpuPercent()); + as.setCpuScaleUpThreshold(autoscaling.cpuScaleUpThreshold()); + as.setCpuProposedReplicas(result.cpuProposedReplicas()); + } as.setProposedReplicas(result.proposedReplicas()); as.setLastScaleTime(lastScaleTimes.get(key)); statuses.put(component, as); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index caa7f17dc340..6ffd24d2dd9b 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -816,6 +816,10 @@ private static String buildJmxExporterConfig(String component) { sb.append("- pattern: 'metrics<>Value'\n"); sb.append(" name: tez_session_$1\n"); sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); break; case "metastore": // HMS API call metrics @@ -825,6 +829,10 @@ private static String buildJmxExporterConfig(String component) { sb.append("- pattern: 'metrics<>Count'\n"); sb.append(" name: hive_metastore_open_connections\n"); sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); break; case "llap": // Only export the executor metrics the autoscaler and drain script need. diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java index 2ec6d5fda0c3..9f0a0a908d7c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -53,7 +53,15 @@ public record AutoscalingSpec( @JsonPropertyDescription("How often (seconds) the operator scrapes JMX metrics from pods. " + "Lower values make autoscaling react faster.") @Default("10") - Integer metricsScrapeIntervalSeconds) { + Integer metricsScrapeIntervalSeconds, + @JsonPropertyDescription("CPU percentage (0-100) that triggers scale-up. " + + "Only applies to HS2 and HMS. Set to 0 to disable CPU-based scaling.") + @Default("90") + Integer cpuScaleUpThreshold, + @JsonPropertyDescription("CPU percentage (0-100) below which scale-down is considered. " + + "Only applies to HS2 and HMS.") + @Default("30") + Integer cpuScaleDownThreshold) { public AutoscalingSpec { enabled = enabled != null ? enabled : false; @@ -63,6 +71,8 @@ public record AutoscalingSpec( scaleDownStabilizationSeconds = scaleDownStabilizationSeconds != null ? scaleDownStabilizationSeconds : 600; gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 3600; metricsScrapeIntervalSeconds = metricsScrapeIntervalSeconds != null ? metricsScrapeIntervalSeconds : 10; + cpuScaleUpThreshold = cpuScaleUpThreshold != null ? cpuScaleUpThreshold : 90; + cpuScaleDownThreshold = cpuScaleDownThreshold != null ? cpuScaleDownThreshold : 30; } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index e4703e8294a7..fc934d955ddb 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -62,6 +62,6 @@ public record HiveServer2Spec( extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 80, 60, 600, 300, 10); + false, 1, 80, 60, 600, 300, 10, 90, 30); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index 79d560104ca7..c058232032a6 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -68,7 +68,7 @@ public record LlapSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 1, 60, 900, 600, 10); + false, 0, 1, 60, 900, 600, 10, 0, 0); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 8a45f545fb1f..5957eb989afc 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -69,7 +69,7 @@ public record MetastoreSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 75, 60, 300, 60, 10); + false, 1, 75, 60, 300, 60, 10, 90, 30); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index 96e6933030c2..f11fd6e417d0 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 0, 60, 600, 120, 10); + false, 0, 0, 60, 600, 120, 10, 0, 0); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java index 8d775b9e857a..5a074b7b222e 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java @@ -31,6 +31,9 @@ public class AutoscalingStatus { private int currentMetricValue; private Integer scaleUpThreshold; + private Double currentCpuPercent; + private Integer cpuScaleUpThreshold; + private Integer cpuProposedReplicas; private int proposedReplicas; private String lastScaleTime; @@ -50,6 +53,30 @@ public void setScaleUpThreshold(Integer scaleUpThreshold) { this.scaleUpThreshold = scaleUpThreshold; } + public Double getCurrentCpuPercent() { + return currentCpuPercent; + } + + public void setCurrentCpuPercent(Double currentCpuPercent) { + this.currentCpuPercent = currentCpuPercent; + } + + public Integer getCpuScaleUpThreshold() { + return cpuScaleUpThreshold; + } + + public void setCpuScaleUpThreshold(Integer cpuScaleUpThreshold) { + this.cpuScaleUpThreshold = cpuScaleUpThreshold; + } + + public Integer getCpuProposedReplicas() { + return cpuProposedReplicas; + } + + public void setCpuProposedReplicas(Integer cpuProposedReplicas) { + this.cpuProposedReplicas = cpuProposedReplicas; + } + public int getProposedReplicas() { return proposedReplicas; } @@ -77,6 +104,9 @@ public boolean equals(Object o) { AutoscalingStatus that = (AutoscalingStatus) o; return currentMetricValue == that.currentMetricValue && Objects.equals(scaleUpThreshold, that.scaleUpThreshold) + && Objects.equals(currentCpuPercent, that.currentCpuPercent) + && Objects.equals(cpuScaleUpThreshold, that.cpuScaleUpThreshold) + && Objects.equals(cpuProposedReplicas, that.cpuProposedReplicas) && proposedReplicas == that.proposedReplicas && Objects.equals(lastScaleTime, that.lastScaleTime); } @@ -84,6 +114,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash(currentMetricValue, scaleUpThreshold, + currentCpuPercent, cpuScaleUpThreshold, cpuProposedReplicas, proposedReplicas, lastScaleTime); } } From 6f36315c252a23abb2eefbb0e21aad3741953bfa Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 9 Jun 2026 20:37:24 +0530 Subject: [PATCH 11/22] Auto Suspend --- packaging/src/kubernetes/README.md | 165 +++++++++ .../crds/hiveclusters.hive.apache.org-v1.yml | 49 ++- .../hive-operator/templates/hivecluster.yaml | 11 + .../kubernetes/helm/hive-operator/values.yaml | 14 + packaging/src/kubernetes/pom.xml | 7 +- .../autoscaling/HiveClusterAutoscaler.java | 13 + .../operator/autoscaling/MetricsScraper.java | 2 +- .../dependent/HiveDependentResource.java | 8 + .../dependent/LlapStatefulSetDependent.java | 2 + .../operator/model/HiveClusterSpec.java | 12 +- .../operator/model/HiveClusterStatus.java | 49 ++- .../operator/model/spec/AutoSuspendSpec.java | 54 +++ .../reconciler/HiveClusterReconciler.java | 329 +++++++++++++++++- 13 files changed, 690 insertions(+), 25 deletions(-) create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index 6d18038078a3..300eaf835120 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -637,6 +637,163 @@ terminates immediately once sessions drain, not after the full grace period. | **LLAP** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | | **TezAM** | 0 | No HS2 sessions (activation gate fails) | HS2 has open sessions (next scrape) | +### Auto-Suspend (Full Cluster Hibernation) + +Auto-suspend goes beyond scale-to-zero — it fully hibernates the **entire** cluster +(including HS2 and HMS) to 0 replicas after a configurable idle timeout. This is +useful for dev/test clusters that should not consume resources when nobody is using +them. + +**Prerequisites:** Auto-suspend requires autoscaling to be enabled on ALL active +components (HS2, LLAP if enabled, TezAM if enabled, and HMS if `includeMetastore=true`). +The operator will not auto-suspend unless it can confirm all components are at their +minimum state. + +**Idle criteria (all must hold simultaneously for `idleTimeoutMinutes`):** + +| Component | Idle Condition | +|-----------|---------------| +| **HS2** | At `minReplicas` with 0 open sessions | +| **HMS** | At `minReplicas` (only checked if `includeMetastore=true`) | +| **LLAP** | At `minReplicas` (default 0) | +| **TezAM** | At `minReplicas` (default 0) | + +**Important:** HS2 can **only** scale to 0 replicas via auto-suspend. Normal +autoscaling always maintains `minReplicas >= 1` for HS2. Auto-suspend is the +only mechanism that overrides this to achieve full hibernation. + +``` + Auto-Suspend Flow + + 1. Autoscaling scales all components to their minReplicas + (HS2≥1, HMS≥1, LLAP/TezAM to configured min) + + 2. Operator detects idle state: + - HS2 has 0 open sessions + - HMS at minReplicas (if includeMetastore=true) + - LLAP/TezAM at minReplicas + + 3. Idle timer starts (status: clusterPhase=Idle, idleSince=) + + 4. After idleTimeoutMinutes (default 15): + - ALL components scaled to 0 (HMS excluded if includeMetastore=false) + - spec.suspend set to true (cluster stays suspended until user wakes it) + - Status: clusterPhase=Suspended, suspendedSince= + + 5. To wake: kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' + All components restored to minReplicas + (HS2/HMS ≥1, LLAP/TezAM ≥1 for immediate usability) + +``` + +**Configuration:** + +```yaml +cluster: + autoSuspend: + enabled: true + idleTimeoutMinutes: 15 # minutes idle before full hibernation + includeMetastore: true # set false to keep HMS running during suspend +``` + +**Manual Suspend/Wake Commands:** + +```bash +# Suspend immediately (bypasses idle timer) +kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":true}}' + +# Wake cluster (restores to minReplicas) +kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' +``` + +Manual suspend works regardless of whether `autoSuspend.enabled` is true — it +immediately scales all components to 0 without waiting for the idle timeout. +When `includeMetastore: false`, HMS stays running even during manual suspend. + +**Observing cluster state:** + +```bash +# Quick view — printer columns show phase and idle time +kubectl get hivecluster +``` +``` +NAME PHASE IDLE (MIN) AGE +hive Idle 12 2h +``` + +```bash +# After suspend triggers +kubectl get hivecluster +``` +``` +NAME PHASE IDLE (MIN) AGE +hive Suspended 2h +``` + +```bash +# Full status (kubectl get hivecluster hive -o yaml) +``` +```yaml +status: + clusterPhase: Suspended + idleSince: "2026-06-08T10:00:00Z" + idleForMinutes: 15 + suspendedSince: "2026-06-08T10:15:00Z" + conditions: + - type: Suspended + status: "True" + reason: AutoSuspend # or ManualSuspend + message: "Cluster suspended after idle timeout" + lastTransitionTime: "2026-06-08T10:15:00Z" +``` + +When the cluster is running normally: +``` +NAME PHASE IDLE (MIN) AGE +hive Running 2h +``` + +**Full example (autoscaling + auto-suspend):** + +```yaml +cluster: + autoSuspend: + enabled: true + idleTimeoutMinutes: 15 + includeMetastore: false # keep HMS running during suspend + + hiveServer2: + replicas: 10 + autoscaling: + enabled: true + minReplicas: 1 + + metastore: + replicas: 6 + autoscaling: + enabled: true + minReplicas: 1 + + llap: + replicas: 8 + autoscaling: + enabled: true + minReplicas: 0 # scales to 0 via normal autoscaling when HS2 idle + + tezAm: + replicas: 10 + autoscaling: + enabled: true + minReplicas: 0 # scales to 0 via normal autoscaling when HS2 idle +``` + +With this configuration, the cluster lifecycle is: +1. Under load → all components scaled up by autoscaler +2. Load drops → autoscaler scales to minReplicas (HS2=1, HMS=1, LLAP=0, TezAM=0) +3. HS2 idle (0 sessions) for 15 minutes → auto-suspend kicks in → HS2, LLAP, TezAM to 0 (HMS stays at minReplicas) +4. `kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}'` → wake → HS2=1, LLAP=1, TezAM=1 +5. User connects → autoscaler detects sessions → scales up as needed + ### CPU-Based Scaling (HS2 and HMS) In addition to the primary metrics (sessions for HS2, API request rate for HMS), @@ -1046,6 +1203,14 @@ setup is needed — simply connect to HS2 and the operator wakes LLAP/TezAM as n | `cluster.tezAm.extraVolumes` | `[]` | Additional volumes for TezAM pods | | `cluster.tezAm.extraVolumeMounts` | `[]` | Additional volume mounts for TezAM containers | +### Auto-Suspend + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster.autoSuspend.enabled` | `false` | Enable full cluster hibernation after idle timeout. Requires autoscaling enabled on all active components (HMS only if `includeMetastore=true`). | +| `cluster.autoSuspend.idleTimeoutMinutes` | `15` | Minutes of idle time (HS2=0 sessions, LLAP/TezAM at minReplicas) before the cluster suspends. | +| `cluster.autoSuspend.includeMetastore` | `true` | Whether HMS participates in auto-suspend. When false, HMS stays at minReplicas during suspend and HMS autoscaling is not required. | + ### Autoscaling (per component) | Value | Default | Description | diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 47f49ddc79d3..913692c99fff 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -13,12 +13,47 @@ spec: singular: hivecluster scope: Namespaced versions: - - name: v1 + - additionalPrinterColumns: + - jsonPath: .status.clusterPhase + name: Phase + priority: 0 + type: string + - jsonPath: .status.idleForMinutes + name: Idle (min) + priority: 0 + type: integer + - jsonPath: .status.suspendedSince + name: Suspended Since + priority: 1 + type: string + name: v1 schema: openAPIV3Schema: properties: spec: properties: + autoSuspend: + description: "Auto-suspend configuration. When enabled and all components\ + \ are idle for the configured timeout, the cluster scales to 0 replicas." + properties: + enabled: + default: false + description: "Whether auto-suspend is enabled. Requires autoscaling\ + \ to be enabled on all active components (HS2, LLAP if enabled,\ + \ TezAM if enabled, and HMS if includeMetastore is true)." + type: boolean + idleTimeoutMinutes: + default: 15 + description: "Minutes of idle time (HS2=0 sessions, LLAP/TezAM\ + \ at minReplicas) before the cluster auto-suspends." + type: integer + includeMetastore: + default: true + description: "Whether Metastore participates in auto-suspend.\ + \ When false, HMS stays at minReplicas during suspend and HMS\ + \ autoscaling is not required for auto-suspend to activate." + type: boolean + type: object envVars: description: "Environment variables injected into all component pods\ \ (e.g., storage credentials, custom JVM options)" @@ -542,6 +577,10 @@ spec: type: string type: object x-kubernetes-preserve-unknown-fields: true + suspend: + description: "When true, the cluster is immediately suspended (all\ + \ components scaled to 0). Set to false to wake a suspended cluster." + type: boolean tezAm: description: Tez Application Master configuration. Enabled by default. properties: @@ -689,6 +728,8 @@ spec: x-kubernetes-preserve-unknown-fields: true status: properties: + clusterPhase: + type: string conditions: items: properties: @@ -736,6 +777,10 @@ spec: readyReplicas: type: integer type: object + idleForMinutes: + type: integer + idleSince: + type: string llap: properties: autoscaling: @@ -798,6 +843,8 @@ spec: type: object observedGeneration: type: integer + suspendedSince: + type: string tezAm: properties: autoscaling: diff --git a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml index 1bfea9c3abbc..6b053ad4ec3b 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -219,4 +219,15 @@ spec: volumeMounts: {{- toYaml .Values.cluster.storage.volumeMounts | nindent 4 }} {{- end }} + + {{- if and .Values.cluster.autoSuspend .Values.cluster.autoSuspend.enabled }} + autoSuspend: + enabled: true + idleTimeoutMinutes: {{ .Values.cluster.autoSuspend.idleTimeoutMinutes | default 15 }} + {{- if hasKey .Values.cluster.autoSuspend "includeMetastore" }} + includeMetastore: {{ .Values.cluster.autoSuspend.includeMetastore }} + {{- end }} + {{- end }} + + suspend: false {{- end }} diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index 88f720d65c7e..99e9b47b6a17 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -101,6 +101,20 @@ cluster: # mountPath: /etc/gcs # readOnly: true + # --------------------------------------------------------------------------- + # AUTO-SUSPEND — fully hibernates the cluster after idle timeout + # --------------------------------------------------------------------------- + # When enabled (requires autoscaling on all active components), the operator + # scales the entire cluster to 0 replicas after all components have been idle + # for idleTimeoutMinutes. Use kubectl patch to manually suspend/wake: + # kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":true}}' + # kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' + autoSuspend: + enabled: false + idleTimeoutMinutes: 15 + # Set to false to keep HMS running during suspend (HMS autoscaling not required) + includeMetastore: true + # --------------------------------------------------------------------------- # METASTORE — defaults to enabled, 2 replicas (HA) # --------------------------------------------------------------------------- diff --git a/packaging/src/kubernetes/pom.xml b/packaging/src/kubernetes/pom.xml index 7fe9937a5eb7..6cb6d6ce0c24 100644 --- a/packaging/src/kubernetes/pom.xml +++ b/packaging/src/kubernetes/pom.xml @@ -65,9 +65,14 @@ ${fabric8.version} provided + + org.slf4j + slf4j-api + 2.0.16 + org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl ${log4j2.version} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index d9989af8e5eb..8394c6932936 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -62,6 +62,15 @@ public static Integer getManagedReplicas(String namespace, String clusterName, S return MANAGED_REPLICAS.get(namespace + "/" + clusterName + "/" + component); } + /** + * Sets the managed replica count for a component. Used by suspend/wake logic + * to override what the autoscaler would normally compute. + */ + public static void setManagedReplicas(String namespace, String clusterName, + String component, int replicas) { + MANAGED_REPLICAS.put(namespace + "/" + clusterName + "/" + component, replicas); + } + private final MetricsScraper scraper; // Key: "namespace/clusterName/component" private final ConcurrentHashMap autoscalers = @@ -73,6 +82,10 @@ public HiveClusterAutoscaler(MetricsScraper scraper) { this.scraper = scraper; } + public MetricsScraper getScraper() { + return scraper; + } + /** * Evaluate all autoscaling-enabled components and return patches and status info. * diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java index c60e9f96d23c..c30702d8c639 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java @@ -98,7 +98,7 @@ public List scrape(String namespace, Map selector) { futures.add(fetchMetricsAsync(podIp) .thenApply(body -> new PodMetrics(podName, PrometheusTextParser.parse(body))) .exceptionally(ex -> { - LOG.warn("Failed to scrape metrics from pod {}: {}", podName, ex.getMessage()); + LOG.debug("Failed to scrape metrics from pod {}: {}", podName, ex.getMessage()); return null; })); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 6ffd24d2dd9b..5005cb9f5963 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -176,6 +176,14 @@ protected R handleCreate(R desired, P primary, Context

context) { */ protected Integer resolveReplicaCount(P primary, Context

context, AutoscalingSpec autoscaling, int staticReplicas, int initialReplicas) { + // Suspended cluster → 0 replicas (dependent resources natively respect suspend). + // Exception: HMS stays running if includeMetastore=false in autoSuspend config. + if (primary instanceof HiveCluster hc && hc.getSpec().suspend()) { + boolean isMetastore = "metastore".equals(getComponentName()); + if (!isMetastore || hc.getSpec().autoSuspend().includeMetastore()) { + return 0; + } + } if (autoscaling == null || !autoscaling.isEnabled()) { return staticReplicas; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 09e68fbf5e70..a1868361b53d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -87,6 +87,8 @@ protected StatefulSet desired(HiveCluster hiveCluster, spec.zookeeper().quorum(), null)); envVars.add(new EnvVar("HIVE_LLAP_DAEMON_SERVICE_HOSTS", llap.serviceHosts(), null)); + envVars.add(new EnvVar("LLAP_LOG4J2_PROPERTIES_FILE_NAME", + "llap-daemon-log4j2.properties", null)); // User-provided env vars (storage credentials, etc.) if (spec.envVars() != null) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java index 1897582bd18e..a6d9c709880c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java @@ -28,6 +28,7 @@ import io.fabric8.kubernetes.api.model.EnvVar; import io.fabric8.kubernetes.api.model.Volume; import io.fabric8.kubernetes.api.model.VolumeMount; +import org.apache.hive.kubernetes.operator.model.spec.AutoSuspendSpec; import org.apache.hive.kubernetes.operator.model.spec.HadoopSpec; import org.apache.hive.kubernetes.operator.model.spec.HiveServer2Spec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; @@ -73,7 +74,13 @@ public record HiveClusterSpec( "Volume mounts added to all component containers " + "(e.g., mounting a GCS key file at /etc/gcs/key.json)") @SchemaFrom(type = Object[].class) @PreserveUnknownFields - List volumeMounts) { + List volumeMounts, + @JsonPropertyDescription("Auto-suspend configuration. When enabled and all components " + + "are idle for the configured timeout, the cluster scales to 0 replicas.") + AutoSuspendSpec autoSuspend, + @JsonPropertyDescription("When true, the cluster is immediately suspended (all components " + + "scaled to 0). Set to false to wake a suspended cluster.") + Boolean suspend) { public HiveClusterSpec { Objects.requireNonNull(zookeeper, @@ -90,5 +97,8 @@ public record HiveClusterSpec( externalJars = externalJars != null ? externalJars : List.of(); volumes = volumes != null ? volumes : List.of(); volumeMounts = volumeMounts != null ? volumeMounts : List.of(); + autoSuspend = autoSuspend != null ? autoSuspend : new AutoSuspendSpec(false, 15, true); + suspend = suspend != null ? suspend : false; } + } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java index d2432dda2246..ea6758309f11 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import io.fabric8.crd.generator.annotation.PrinterColumn; import io.fabric8.kubernetes.api.model.Condition; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; @@ -33,6 +34,13 @@ public class HiveClusterStatus { private ComponentStatus llap; private ComponentStatus tezAm; private Long observedGeneration; + @PrinterColumn(name = "Phase") + private String clusterPhase; + private String idleSince; + @PrinterColumn(name = "Idle (min)") + private Integer idleForMinutes; + @PrinterColumn(name = "Suspended Since", priority = 1) + private String suspendedSince; public List getConditions() { return conditions; @@ -82,6 +90,38 @@ public void setObservedGeneration(Long observedGeneration) { this.observedGeneration = observedGeneration; } + public String getClusterPhase() { + return clusterPhase; + } + + public void setClusterPhase(String clusterPhase) { + this.clusterPhase = clusterPhase; + } + + public String getIdleSince() { + return idleSince; + } + + public void setIdleSince(String idleSince) { + this.idleSince = idleSince; + } + + public Integer getIdleForMinutes() { + return idleForMinutes; + } + + public void setIdleForMinutes(Integer idleForMinutes) { + this.idleForMinutes = idleForMinutes; + } + + public String getSuspendedSince() { + return suspendedSince; + } + + public void setSuspendedSince(String suspendedSince) { + this.suspendedSince = suspendedSince; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -96,11 +136,16 @@ public boolean equals(Object o) { java.util.Objects.equals(metastore, that.metastore) && java.util.Objects.equals(hiveServer2, that.hiveServer2) && java.util.Objects.equals(llap, that.llap) && - java.util.Objects.equals(tezAm, that.tezAm); + java.util.Objects.equals(tezAm, that.tezAm) && + java.util.Objects.equals(clusterPhase, that.clusterPhase) && + java.util.Objects.equals(idleSince, that.idleSince) && + java.util.Objects.equals(idleForMinutes, that.idleForMinutes) && + java.util.Objects.equals(suspendedSince, that.suspendedSince); } @Override public int hashCode() { - return java.util.Objects.hash(conditions, metastore, hiveServer2, llap, tezAm, observedGeneration); + return java.util.Objects.hash(conditions, metastore, hiveServer2, llap, tezAm, + observedGeneration, clusterPhase, idleSince, idleForMinutes, suspendedSince); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java new file mode 100644 index 000000000000..196d57e8bc86 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java @@ -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.hive.kubernetes.operator.model.spec; + +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import io.fabric8.generator.annotation.Default; + +/** + * Auto-suspend configuration. When enabled and all components are idle for the + * configured timeout, the operator scales the entire cluster to 0 replicas. + * Requires autoscaling to be enabled on all active components. + */ +public record AutoSuspendSpec( + @JsonPropertyDescription("Whether auto-suspend is enabled. Requires autoscaling " + + "to be enabled on all active components (HS2, LLAP if enabled, TezAM if enabled, " + + "and HMS if includeMetastore is true).") + @Default("false") + Boolean enabled, + @JsonPropertyDescription("Minutes of idle time (HS2=0 sessions, LLAP/TezAM at minReplicas) " + + "before the cluster auto-suspends.") + @Default("15") + Integer idleTimeoutMinutes, + @JsonPropertyDescription("Whether Metastore participates in auto-suspend. " + + "When false, HMS stays at minReplicas during suspend and HMS autoscaling " + + "is not required for auto-suspend to activate.") + @Default("true") + Boolean includeMetastore) { + + public AutoSuspendSpec { + enabled = enabled != null ? enabled : false; + idleTimeoutMinutes = idleTimeoutMinutes != null ? idleTimeoutMinutes : 15; + includeMetastore = includeMetastore != null ? includeMetastore : true; + } + + public boolean isEnabled() { + return enabled; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 96110762dd1d..50d6714565b2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -39,9 +39,11 @@ import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; import org.apache.hive.kubernetes.operator.autoscaling.HiveClusterAutoscaler; import org.apache.hive.kubernetes.operator.autoscaling.MetricsScraper; +import org.apache.hive.kubernetes.operator.autoscaling.PodMetrics; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.HiveClusterStatus; +import org.apache.hive.kubernetes.operator.model.spec.AutoSuspendSpec; import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; import org.slf4j.Logger; @@ -68,30 +70,101 @@ public UpdateControl reconcile(HiveCluster resource, Context entry : eval.patches().entrySet()) { patchReplicas(client, resource, entry.getKey(), entry.getValue()); } - // Set autoscaling status on each component applyAutoscalingStatuses(newStatus, eval.statuses()); - // Always patch status when autoscaling is active (metrics change each scrape) - int interval = getMinScrapeInterval(resource.getSpec()); - resource.setStatus(newStatus); - return UpdateControl.patchStatus(resource) - .rescheduleAfter(Duration.ofSeconds(interval)); + rescheduleSeconds = getMinScrapeInterval(resource.getSpec()); } - if (!statusChanged) { + // --- Single exit point for status update --- + boolean statusNowChanged = !statusEqualsIgnoringTimestamps(existingStatus, newStatus); + if (!statusNowChanged && rescheduleSeconds == 0) { return UpdateControl.noUpdate(); } - resource.setStatus(newStatus); + if (rescheduleSeconds > 0) { + return UpdateControl.patchStatus(resource) + .rescheduleAfter(Duration.ofSeconds(rescheduleSeconds)); + } return UpdateControl.patchStatus(resource); } @@ -417,12 +490,230 @@ private void patchReplicas(KubernetesClient client, HiveCluster resource, String component, int replicas) { String namespace = resource.getMetadata().getNamespace(); String workloadName = resource.getMetadata().getName() + "-" + component; - if ("llap".equals(component) || "tezam".equals(component)) { - client.apps().statefulSets().inNamespace(namespace).withName(workloadName).scale(replicas); - LOG.info("Scaled StatefulSet {}/{} to {} replicas", namespace, workloadName, replicas); - } else { - client.apps().deployments().inNamespace(namespace).withName(workloadName).scale(replicas); - LOG.info("Scaled Deployment {}/{} to {} replicas", namespace, workloadName, replicas); + try { + if ("llap".equals(component) || "tezam".equals(component)) { + client.apps().statefulSets().inNamespace(namespace).withName(workloadName).scale(replicas); + } else { + client.apps().deployments().inNamespace(namespace).withName(workloadName).scale(replicas); + } + LOG.info("Scaled {}/{} to {} replicas", namespace, workloadName, replicas); + } catch (Exception e) { + LOG.debug("Could not scale {}/{}: {}", namespace, workloadName, e.getMessage()); + } + } + + private void patchSuspendSpec(KubernetesClient client, HiveCluster resource, boolean suspend) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + client.resources(HiveCluster.class).inNamespace(ns).withName(name) + .edit(hc -> { + // Records are immutable so we build a new spec with the updated suspend value + HiveClusterSpec oldSpec = hc.getSpec(); + HiveClusterSpec newSpec = new HiveClusterSpec( + oldSpec.image(), oldSpec.imagePullPolicy(), oldSpec.metastore(), + oldSpec.hiveServer2(), oldSpec.llap(), oldSpec.tezAm(), oldSpec.zookeeper(), + oldSpec.hadoop(), oldSpec.envVars(), oldSpec.externalJars(), + oldSpec.volumes(), oldSpec.volumeMounts(), oldSpec.autoSuspend(), suspend); + hc.setSpec(newSpec); + return hc; + }); + LOG.info("Patched spec.suspend={} on {}/{}", suspend, ns, name); + } + + // --- Auto-Suspend / Wake --- + + enum SuspendAction { RUNNING, IDLE_START, IDLE_WAITING, SUSPEND_NOW, STAY_SUSPENDED, WAKE } + + private SuspendAction evaluateSuspendState(HiveCluster resource, + HiveClusterStatus existingStatus, KubernetesClient client) { + + // 1. Manual suspend: spec.suspend = true → suspend immediately + if (resource.getSpec().suspend()) { + if (existingStatus != null && "Suspended".equals(existingStatus.getClusterPhase())) { + return SuspendAction.STAY_SUSPENDED; + } + return SuspendAction.SUSPEND_NOW; + } + + // 2. Currently suspended and spec.suspend = false → wake + if (existingStatus != null && "Suspended".equals(existingStatus.getClusterPhase())) { + return SuspendAction.WAKE; + } + + // 3. Auto-suspend evaluation (only if enabled and all autoscaling is on) + AutoSuspendSpec autoSuspend = resource.getSpec().autoSuspend(); + if (!autoSuspend.isEnabled()) { + LOG.debug("Auto-suspend disabled"); + return SuspendAction.RUNNING; + } + if (!allAutoscalingEnabled(resource.getSpec())) { + LOG.debug("Auto-suspend skipped: not all components have autoscaling enabled"); + return SuspendAction.RUNNING; + } + + // 4. Check idle conditions + boolean allIdle = isClusterIdle(resource, existingStatus, client); + if (!allIdle) { + return SuspendAction.RUNNING; + } + + // 5. Check idle duration + String idleSince = existingStatus != null ? existingStatus.getIdleSince() : null; + if (idleSince == null) { + return SuspendAction.IDLE_START; + } + + Instant idleStart = Instant.parse(idleSince); + if (Duration.between(idleStart, Instant.now()).toMinutes() >= autoSuspend.idleTimeoutMinutes()) { + return SuspendAction.SUSPEND_NOW; + } + + return SuspendAction.IDLE_WAITING; + } + + + private boolean isClusterIdle(HiveCluster resource, HiveClusterStatus existingStatus, + KubernetesClient client) { + HiveClusterSpec spec = resource.getSpec(); + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + + // All components must be at minReplicas + if (spec.llap().isEnabled() + && !isAtMinReplicas(client, ns, name + "-llap", true, + spec.llap().autoscaling().minReplicas())) { + return false; + } + if (spec.tezAm().isEnabled() + && !isAtMinReplicas(client, ns, name + "-tezam", true, + spec.tezAm().autoscaling().minReplicas())) { + return false; + } + if (!isAtMinReplicas(client, ns, name + "-hiveserver2", false, + Math.max(1, spec.hiveServer2().autoscaling().minReplicas()))) { + return false; + } + + // HS2 must have 0 open sessions. + // If metrics scrape fails (empty list), assume NOT idle to prevent accidental suspend. + HiveClusterAutoscaler scaler = getOrCreateAutoscaler(client); + List hs2Metrics = scaler.scrapeHs2Metrics(resource); + if (hs2Metrics.isEmpty()) { + LOG.debug("Idle check: HS2 metrics unavailable, assuming not idle"); + return false; + } + int totalSessions = hs2Metrics.stream() + .mapToInt(pm -> pm.metrics().getOrDefault("hs2_open_sessions", 0.0).intValue()) + .sum(); + if (totalSessions > 0) { + LOG.debug("Idle check failed: HS2 has {} open sessions", totalSessions); + return false; + } + + // HMS must be at minReplicas (only checked if includeMetastore=true) + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore() + && !isAtMinReplicas(client, ns, name + "-metastore", false, + Math.max(1, spec.metastore().autoscaling().minReplicas()))) { + return false; + } + + return true; + } + + /** Returns true if the workload is absent or its replicas <= minReplicas. */ + private boolean isAtMinReplicas(KubernetesClient client, String ns, + String workloadName, boolean statefulSet, int minReplicas) { + try { + Integer currentReplicas = null; + if (statefulSet) { + var ss = client.apps().statefulSets().inNamespace(ns).withName(workloadName).get(); + if (ss != null && ss.getSpec() != null) { + currentReplicas = ss.getSpec().getReplicas(); + } + } else { + var deploy = client.apps().deployments().inNamespace(ns).withName(workloadName).get(); + if (deploy != null && deploy.getSpec() != null) { + currentReplicas = deploy.getSpec().getReplicas(); + } + } + if (currentReplicas != null && currentReplicas > minReplicas) { + LOG.debug("Idle check failed: {} replicas {} > min {}", workloadName, currentReplicas, minReplicas); + return false; + } + return true; + } catch (Exception e) { + LOG.debug("Idle check: could not read {}: {}", workloadName, e.getMessage()); + return true; } } + + private void suspendCluster(HiveCluster resource, KubernetesClient client) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + HiveClusterSpec spec = resource.getSpec(); + + // Set MANAGED_REPLICAS to 0 so autoscaler doesn't fight the suspend. + // Actual scaling to 0 is handled by the DependentResources which check + // spec.suspend() in resolveReplicaCount(). + HiveClusterAutoscaler.setManagedReplicas(ns, name, "hiveserver2", 0); + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, "metastore", 0); + } + if (spec.llap().isEnabled()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, "llap", 0); + } + if (spec.tezAm().isEnabled()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, "tezam", 0); + } + + LOG.info("Cluster {}/{} suspended", ns, name); + } + + private void wakeCluster(HiveCluster resource, KubernetesClient client) { + HiveClusterSpec spec = resource.getSpec(); + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + + // Set MANAGED_REPLICAS to wake values. The JOSDK workflow will recreate + // the dependent resources (Deployments/StatefulSets) on the next reconcile + // and use these values for spec.replicas. We don't call patchReplicas() + // because the workloads may have been garbage-collected while suspended. + int hs2Min = Math.max(1, spec.hiveServer2().autoscaling().minReplicas()); + HiveClusterAutoscaler.setManagedReplicas(ns, name, "hiveserver2", hs2Min); + + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { + int hmsMin = Math.max(1, spec.metastore().autoscaling().minReplicas()); + HiveClusterAutoscaler.setManagedReplicas(ns, name, "metastore", hmsMin); + } + + if (spec.llap().isEnabled()) { + int llapWake = spec.llap().autoscaling().minReplicas(); + HiveClusterAutoscaler.setManagedReplicas(ns, name, "llap", llapWake); + } + + if (spec.tezAm().isEnabled()) { + int tezWake = spec.tezAm().autoscaling().minReplicas(); + HiveClusterAutoscaler.setManagedReplicas(ns, name, "tezam", tezWake); + } + + LOG.info("Cluster {}/{} woken up — restored to minReplicas", ns, name); + } + + private static boolean allAutoscalingEnabled(HiveClusterSpec spec) { + if (!spec.hiveServer2().autoscaling().isEnabled()) { + return false; + } + // Skip HMS check if includeMetastore=false (HMS doesn't participate in suspend) + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore() + && !spec.metastore().autoscaling().isEnabled()) { + return false; + } + if (spec.llap().isEnabled() && !spec.llap().autoscaling().isEnabled()) { + return false; + } + if (spec.tezAm().isEnabled() && !spec.tezAm().autoscaling().isEnabled()) { + return false; + } + return true; + } } From c25b9c5fede482b85afcb88649212abc147acc41 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 10 Jun 2026 04:44:12 +0530 Subject: [PATCH 12/22] Refactor --- .../autoscaling/ComponentAutoscaler.java | 4 +- .../autoscaling/HiveClusterAutoscaler.java | 28 ++++--- .../autoscaling/PrometheusTextParser.java | 62 ++++----------- .../dependent/HiveConfigMapDependent.java | 7 +- .../dependent/HiveDependentResource.java | 12 +-- .../operator/dependent/HivePdbDependent.java | 9 ++- .../HiveServer2DeploymentDependent.java | 8 +- .../dependent/HiveServiceDependent.java | 8 +- .../dependent/LlapStatefulSetDependent.java | 9 ++- .../MetastoreDeploymentDependent.java | 8 +- .../dependent/SchemaInitJobDependent.java | 3 +- .../dependent/TezAmStatefulSetDependent.java | 9 ++- .../reconciler/HiveClusterReconciler.java | 43 +++++----- .../operator/reconciler/HiveWorkflowSpec.java | 79 ++++++++++++------- .../kubernetes/operator/util/ConfigUtils.java | 6 ++ 15 files changed, 148 insertions(+), 147 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java index d8aa95d32dc3..6824b0e8cb79 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +54,8 @@ public record EvaluationResult(int rawMetricValue, double cpuPercent, public ComponentAutoscaler(String component, ScalingStrategy strategy) { this.component = component; this.strategy = strategy; - this.cpuScalingApplicable = "hiveserver2".equals(component) || "metastore".equals(component); + this.cpuScalingApplicable = ConfigUtils.COMPONENT_HIVESERVER2.equals(component) + || ConfigUtils.COMPONENT_METASTORE.equals(component); } /** Whether the underlying strategy uses scaleUpThreshold for scaling decisions. */ diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 8394c6932936..e345102482d1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -29,6 +29,7 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,32 +103,32 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie // HiveServer2 if (spec.hiveServer2().autoscaling().isEnabled()) { - Map hs2Selector = Labels.selectorForComponent(cluster, "hiveserver2"); + Map hs2Selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); List hs2Metrics = scraper.scrape(namespace, hs2Selector); updatePodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); evaluateComponent(cluster, client, namespace, clusterName, - "hiveserver2", spec.hiveServer2().autoscaling(), + ConfigUtils.COMPONENT_HIVESERVER2, spec.hiveServer2().autoscaling(), spec.hiveServer2().replicas(), patches, statuses, hs2Metrics); } // Metastore if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { evaluateComponent(cluster, client, namespace, clusterName, - "metastore", spec.metastore().autoscaling(), + ConfigUtils.COMPONENT_METASTORE, spec.metastore().autoscaling(), spec.metastore().replicas(), patches, statuses); } // LLAP if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { evaluateComponent(cluster, client, namespace, clusterName, - "llap", spec.llap().autoscaling(), + ConfigUtils.COMPONENT_LLAP, spec.llap().autoscaling(), spec.llap().replicas(), patches, statuses); } // TezAM if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { evaluateComponent(cluster, client, namespace, clusterName, - "tezam", spec.tezAm().autoscaling(), + ConfigUtils.COMPONENT_TEZAM, spec.tezAm().autoscaling(), spec.tezAm().replicas(), patches, statuses); } @@ -139,7 +140,7 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie */ public List scrapeHs2Metrics(HiveCluster cluster) { String namespace = cluster.getMetadata().getNamespace(); - Map selector = Labels.selectorForComponent(cluster, "hiveserver2"); + Map selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); return scraper.scrape(namespace, selector); } @@ -171,7 +172,8 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), // not their own pod metrics. Allow evaluation even with 0 own pods. - boolean usesHs2Activation = component.equals("llap") || component.equals("tezam"); + boolean usesHs2Activation = ConfigUtils.COMPONENT_LLAP.equals(component) + || ConfigUtils.COMPONENT_TEZAM.equals(component); if (metrics.isEmpty() && !usesHs2Activation) { LOG.debug("[{}] No ready pods to scrape, skipping", component); @@ -196,7 +198,7 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, as.setScaleUpThreshold(autoscaling.scaleUpThreshold()); } // CPU metrics (only for HS2 and HMS — LLAP/TezAM don't use CPU-based scaling) - if (("hiveserver2".equals(component) || "metastore".equals(component)) + if ((ConfigUtils.COMPONENT_HIVESERVER2.equals(component) || ConfigUtils.COMPONENT_METASTORE.equals(component)) && autoscaling.cpuScaleUpThreshold() > 0) { as.setCurrentCpuPercent(result.cpuPercent()); as.setCpuScaleUpThreshold(autoscaling.cpuScaleUpThreshold()); @@ -218,10 +220,10 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, private ScalingStrategy createStrategy(String component, HiveCluster cluster) { return switch (component) { - case "hiveserver2" -> new HiveServer2ScalingStrategy(); - case "metastore" -> new MetastoreScalingStrategy(); - case "llap" -> new LlapScalingStrategy(this, cluster); - case "tezam" -> new TezAmScalingStrategy(this, cluster); + case ConfigUtils.COMPONENT_HIVESERVER2 -> new HiveServer2ScalingStrategy(); + case ConfigUtils.COMPONENT_METASTORE -> new MetastoreScalingStrategy(); + case ConfigUtils.COMPONENT_LLAP -> new LlapScalingStrategy(this, cluster); + case ConfigUtils.COMPONENT_TEZAM -> new TezAmScalingStrategy(this, cluster); default -> throw new IllegalArgumentException("Unknown component: " + component); }; } @@ -229,7 +231,7 @@ private ScalingStrategy createStrategy(String component, HiveCluster cluster) { private int getCurrentReplicas(KubernetesClient client, String namespace, String clusterName, String component) { String workloadName = clusterName + "-" + component; - if ("llap".equals(component) || "tezam".equals(component)) { + if (ConfigUtils.COMPONENT_LLAP.equals(component) || ConfigUtils.COMPONENT_TEZAM.equals(component)) { var ss = client.apps().statefulSets() .inNamespace(namespace).withName(workloadName).get(); return ss != null && ss.getSpec().getReplicas() != null ? ss.getSpec().getReplicas() : 0; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java index 3f8ab6527703..8940a80b1934 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -36,53 +36,11 @@ private PrometheusTextParser() { /** * Parse Prometheus text format into metric-name → value map. - * Lines with labels are keyed as "metric_name{labels}" to preserve identity. - * Duplicate metric names (e.g. from multiple label sets) are summed. + * Labels are stripped from keys; duplicate metric names (from multiple + * label sets) are summed. */ public static Map parse(String body) { - Map result = new HashMap<>(); - if (body == null || body.isEmpty()) { - return result; - } - try (BufferedReader reader = new BufferedReader(new StringReader(body))) { - String line; - while ((line = reader.readLine()) != null) { - if (line.isEmpty() || line.charAt(0) == '#') { - continue; - } - String metricKey; - String valuePart; - int braceStart = line.indexOf('{'); - if (braceStart >= 0) { - int braceEnd = line.indexOf('}', braceStart); - if (braceEnd < 0) { - continue; - } - metricKey = line.substring(0, braceStart); - valuePart = line.substring(braceEnd + 1).trim(); - } else { - int spaceIdx = line.indexOf(' '); - if (spaceIdx < 0) { - continue; - } - metricKey = line.substring(0, spaceIdx); - valuePart = line.substring(spaceIdx + 1).trim(); - } - int spaceInValue = valuePart.indexOf(' '); - if (spaceInValue > 0) { - valuePart = valuePart.substring(0, spaceInValue); - } - try { - double value = Double.parseDouble(valuePart); - result.merge(metricKey, value, Double::sum); - } catch (NumberFormatException e) { - // Skip NaN, +Inf, -Inf, or malformed values - } - } - } catch (IOException e) { - // StringReader does not throw IOException - } - return result; + return doParse(body, false); } /** @@ -90,6 +48,10 @@ public static Map parse(String body) { * Key format: "metric_name{label=value,...}" */ public static Map parseWithLabels(String body) { + return doParse(body, true); + } + + private static Map doParse(String body, boolean keepLabels) { Map result = new HashMap<>(); if (body == null || body.isEmpty()) { return result; @@ -108,7 +70,7 @@ public static Map parseWithLabels(String body) { if (braceEnd < 0) { continue; } - metricKey = line.substring(0, braceEnd + 1); + metricKey = keepLabels ? line.substring(0, braceEnd + 1) : line.substring(0, braceStart); valuePart = line.substring(braceEnd + 1).trim(); } else { int spaceIdx = line.indexOf(' '); @@ -124,9 +86,13 @@ public static Map parseWithLabels(String body) { } try { double value = Double.parseDouble(valuePart); - result.put(metricKey, value); + if (keepLabels) { + result.put(metricKey, value); + } else { + result.merge(metricKey, value, Double::sum); + } } catch (NumberFormatException e) { - // Skip + // Skip NaN, +Inf, -Inf, or malformed values } } } catch (IOException e) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java index 935b47e094cb..2ca7f87232dd 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java @@ -26,6 +26,7 @@ import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -93,7 +94,7 @@ public static String resourceName(HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) public static class Metastore extends HiveConfigMapDependent { public Metastore() { - super("metastore", "metastore-config"); + super(ConfigUtils.COMPONENT_METASTORE, "metastore-config"); } @Override @@ -114,7 +115,7 @@ public static String resourceName(HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) public static class HiveServer2 extends HiveConfigMapDependent { public HiveServer2() { - super("hiveserver2", "hiveserver2-config"); + super(ConfigUtils.COMPONENT_HIVESERVER2, "hiveserver2-config"); } @Override @@ -137,7 +138,7 @@ public static String resourceName(HiveCluster hiveCluster) { + "app.kubernetes.io/managed-by=hive-kubernetes-operator")) public static class Llap extends HiveConfigMapDependent { public Llap() { - super("llap", "llap-config"); + super(ConfigUtils.COMPONENT_LLAP, "llap-config"); } @Override diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 5005cb9f5963..1728dbb33f91 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -179,7 +179,7 @@ protected Integer resolveReplicaCount(P primary, Context

context, // Suspended cluster → 0 replicas (dependent resources natively respect suspend). // Exception: HMS stays running if includeMetastore=false in autoSuspend config. if (primary instanceof HiveCluster hc && hc.getSpec().suspend()) { - boolean isMetastore = "metastore".equals(getComponentName()); + boolean isMetastore = ConfigUtils.COMPONENT_METASTORE.equals(getComponentName()); if (!isMetastore || hc.getSpec().autoSuspend().includeMetastore()) { return 0; } @@ -785,7 +785,7 @@ protected static void addJmxExporter( // LLAP uses LLAP_DAEMON_OPTS (its startup script ignores SERVICE_OPTS). String agentArg = String.format("-javaagent:%s=%d:%s", JMX_EXPORTER_JAR, ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT, JMX_EXPORTER_CONFIG); - String optsEnvVar = "llap".equals(component) ? "LLAP_DAEMON_OPTS" : "SERVICE_OPTS"; + String optsEnvVar = ConfigUtils.COMPONENT_LLAP.equals(component) ? "LLAP_DAEMON_OPTS" : "SERVICE_OPTS"; boolean found = false; for (int i = 0; i < envVars.size(); i++) { if (optsEnvVar.equals(envVars.get(i).getName())) { @@ -812,7 +812,7 @@ private static String buildJmxExporterConfig(String component) { sb.append("rules:\n"); switch (component) { - case "hiveserver2": + case ConfigUtils.COMPONENT_HIVESERVER2: // HS2 session and operation metrics sb.append("- pattern: 'metrics<>Value'\n"); sb.append(" name: hs2_$1\n"); @@ -829,7 +829,7 @@ private static String buildJmxExporterConfig(String component) { sb.append(" name: jvm_process_cpu_load\n"); sb.append(" type: GAUGE\n"); break; - case "metastore": + case ConfigUtils.COMPONENT_METASTORE: // HMS API call metrics sb.append("- pattern: 'metrics<>Count'\n"); sb.append(" name: api_$1_total\n"); @@ -842,7 +842,7 @@ private static String buildJmxExporterConfig(String component) { sb.append(" name: jvm_process_cpu_load\n"); sb.append(" type: GAUGE\n"); break; - case "llap": + case ConfigUtils.COMPONENT_LLAP: // Only export the executor metrics the autoscaler and drain script need. // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, // causing CPU spikes and GC pressure on the LLAP JVM. @@ -861,7 +861,7 @@ private static String buildJmxExporterConfig(String component) { sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); sb.append(" type: GAUGE\n"); break; - case "tezam": + case ConfigUtils.COMPONENT_TEZAM: // TezAM DAG execution metrics sb.append("- pattern: 'Hadoop<>(.+)'\n"); sb.append(" name: tez_am_$1\n"); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java index e19413341c9b..fa1f2abbdc9c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -25,6 +25,7 @@ import io.javaoperatorsdk.operator.api.config.informer.Informer; import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; /** @@ -75,7 +76,7 @@ protected PodDisruptionBudget desired(HiveCluster hiveCluster, ) public static class HiveServer2 extends HivePdbDependent { public HiveServer2() { - super("hiveserver2"); + super(ConfigUtils.COMPONENT_HIVESERVER2); } } @@ -85,7 +86,7 @@ public HiveServer2() { ) public static class Metastore extends HivePdbDependent { public Metastore() { - super("metastore"); + super(ConfigUtils.COMPONENT_METASTORE); } } @@ -95,7 +96,7 @@ public Metastore() { ) public static class Llap extends HivePdbDependent { public Llap() { - super("llap"); + super(ConfigUtils.COMPONENT_LLAP); } } @@ -105,7 +106,7 @@ public Llap() { ) public static class TezAm extends HivePdbDependent { public TezAm() { - super("tezam"); + super(ConfigUtils.COMPONENT_TEZAM); } } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index 4ce67ae9f6d4..d7d81a3ce2a1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -49,7 +49,7 @@ public class HiveServer2DeploymentDependent extends HiveDependentResource { - public static final String COMPONENT = "hiveserver2"; + public static final String COMPONENT = ConfigUtils.COMPONENT_HIVESERVER2; private static final String SCRATCH_MOUNT_PATH = "/opt/hive/scratch"; public HiveServer2DeploymentDependent() { @@ -76,7 +76,7 @@ protected Deployment desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "hiveserver2", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("TEZ_AM_EXTERNAL_ID", "tez-session-hs2", null)); @@ -240,13 +240,13 @@ protected Deployment desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "hiveserver2") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("hiveserver2") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java index edd048e8a322..55fe15e182d2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -74,7 +74,7 @@ protected Service desired(HiveCluster hiveCluster, ) public static class HiveServer2 extends HiveServiceDependent { public HiveServer2() { - super("hiveserver2"); + super(ConfigUtils.COMPONENT_HIVESERVER2); } @Override @@ -109,7 +109,7 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { ) public static class Metastore extends HiveServiceDependent { public Metastore() { - super("metastore"); + super(ConfigUtils.COMPONENT_METASTORE); } @Override @@ -136,7 +136,7 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { ) public static class Llap extends HiveServiceDependent { public Llap() { - super("llap"); + super(ConfigUtils.COMPONENT_LLAP); } @Override @@ -160,7 +160,7 @@ protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { ) public static class TezAm extends HiveServiceDependent { public TezAm() { - super("tezam"); + super(ConfigUtils.COMPONENT_TEZAM); } @Override diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index a1868361b53d..dad4ee6500d6 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -36,6 +36,7 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -51,7 +52,7 @@ public class LlapStatefulSetDependent extends HiveDependentResource { - public static final String COMPONENT = "llap"; + public static final String COMPONENT = ConfigUtils.COMPONENT_LLAP; public LlapStatefulSetDependent() { super(StatefulSet.class); @@ -77,7 +78,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "llap", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("LLAP_MEMORY_MB", String.valueOf(llap.memoryMb()), null)); @@ -161,13 +162,13 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "llap") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("llap") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index ac8ba55df4a4..9784d1df0c76 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -51,7 +51,7 @@ public class MetastoreDeploymentDependent extends HiveDependentResource { - public static final String COMPONENT = "metastore"; + public static final String COMPONENT = ConfigUtils.COMPONENT_METASTORE; public MetastoreDeploymentDependent() { super(Deployment.class); @@ -77,7 +77,7 @@ protected Deployment desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "metastore", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.addAll(buildDbEnvVars(db)); if (spec.envVars() != null) { @@ -153,13 +153,13 @@ protected Deployment desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "metastore") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("metastore") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java index 25d0eb39a0f9..fb4b588401c9 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java @@ -34,6 +34,7 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.DatabaseConfig; import org.apache.hive.kubernetes.operator.model.spec.SecretKeyRef; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; /** @@ -66,7 +67,7 @@ protected Job desired(HiveCluster hiveCluster, DatabaseConfig db = spec.metastore().database(); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "metastore", null)); + envVars.add(new EnvVar("SERVICE_NAME", ConfigUtils.COMPONENT_METASTORE, null)); envVars.add(new EnvVar("IS_RESUME", "false", null)); envVars.add(new EnvVar("HIVE_CUSTOM_CONF_DIR", CONF_MOUNT_PATH, null)); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index c2c3352473b1..fd81af716e6a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -34,6 +34,7 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.TezAmSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -52,7 +53,7 @@ public class TezAmStatefulSetDependent extends HiveDependentResource { - public static final String COMPONENT = "tezam"; + public static final String COMPONENT = ConfigUtils.COMPONENT_TEZAM; private static final String SCRATCH_MOUNT_PATH = "/opt/hive/scratch"; public TezAmStatefulSetDependent() { @@ -79,7 +80,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "tezam", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("HIVE_ZOOKEEPER_QUORUM", spec.zookeeper().quorum(), null)); @@ -155,13 +156,13 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "tezam") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("tezam") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 50d6714565b2..38b69bdd42fe 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -46,6 +46,7 @@ import org.apache.hive.kubernetes.operator.model.spec.AutoSuspendSpec; import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -430,17 +431,17 @@ private boolean conditionsEqualIgnoringTime(List a, List b private void applyAutoscalingStatuses(HiveClusterStatus status, Map statuses) { - if (statuses.containsKey("hiveserver2") && status.getHiveServer2() != null) { - status.getHiveServer2().setAutoscaling(statuses.get("hiveserver2")); + if (statuses.containsKey(ConfigUtils.COMPONENT_HIVESERVER2) && status.getHiveServer2() != null) { + status.getHiveServer2().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_HIVESERVER2)); } - if (statuses.containsKey("metastore") && status.getMetastore() != null) { - status.getMetastore().setAutoscaling(statuses.get("metastore")); + if (statuses.containsKey(ConfigUtils.COMPONENT_METASTORE) && status.getMetastore() != null) { + status.getMetastore().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_METASTORE)); } - if (statuses.containsKey("llap") && status.getLlap() != null) { - status.getLlap().setAutoscaling(statuses.get("llap")); + if (statuses.containsKey(ConfigUtils.COMPONENT_LLAP) && status.getLlap() != null) { + status.getLlap().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_LLAP)); } - if (statuses.containsKey("tezam") && status.getTezAm() != null) { - status.getTezAm().setAutoscaling(statuses.get("tezam")); + if (statuses.containsKey(ConfigUtils.COMPONENT_TEZAM) && status.getTezAm() != null) { + status.getTezAm().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_TEZAM)); } } @@ -491,7 +492,7 @@ private void patchReplicas(KubernetesClient client, HiveCluster resource, String namespace = resource.getMetadata().getNamespace(); String workloadName = resource.getMetadata().getName() + "-" + component; try { - if ("llap".equals(component) || "tezam".equals(component)) { + if (ConfigUtils.COMPONENT_LLAP.equals(component) || ConfigUtils.COMPONENT_TEZAM.equals(component)) { client.apps().statefulSets().inNamespace(namespace).withName(workloadName).scale(replicas); } else { client.apps().deployments().inNamespace(namespace).withName(workloadName).scale(replicas); @@ -580,16 +581,16 @@ private boolean isClusterIdle(HiveCluster resource, HiveClusterStatus existingSt // All components must be at minReplicas if (spec.llap().isEnabled() - && !isAtMinReplicas(client, ns, name + "-llap", true, + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_LLAP, true, spec.llap().autoscaling().minReplicas())) { return false; } if (spec.tezAm().isEnabled() - && !isAtMinReplicas(client, ns, name + "-tezam", true, + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_TEZAM, true, spec.tezAm().autoscaling().minReplicas())) { return false; } - if (!isAtMinReplicas(client, ns, name + "-hiveserver2", false, + if (!isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_HIVESERVER2, false, Math.max(1, spec.hiveServer2().autoscaling().minReplicas()))) { return false; } @@ -612,7 +613,7 @@ private boolean isClusterIdle(HiveCluster resource, HiveClusterStatus existingSt // HMS must be at minReplicas (only checked if includeMetastore=true) if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore() - && !isAtMinReplicas(client, ns, name + "-metastore", false, + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_METASTORE, false, Math.max(1, spec.metastore().autoscaling().minReplicas()))) { return false; } @@ -655,15 +656,15 @@ private void suspendCluster(HiveCluster resource, KubernetesClient client) { // Set MANAGED_REPLICAS to 0 so autoscaler doesn't fight the suspend. // Actual scaling to 0 is handled by the DependentResources which check // spec.suspend() in resolveReplicaCount(). - HiveClusterAutoscaler.setManagedReplicas(ns, name, "hiveserver2", 0); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_HIVESERVER2, 0); if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { - HiveClusterAutoscaler.setManagedReplicas(ns, name, "metastore", 0); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_METASTORE, 0); } if (spec.llap().isEnabled()) { - HiveClusterAutoscaler.setManagedReplicas(ns, name, "llap", 0); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_LLAP, 0); } if (spec.tezAm().isEnabled()) { - HiveClusterAutoscaler.setManagedReplicas(ns, name, "tezam", 0); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_TEZAM, 0); } LOG.info("Cluster {}/{} suspended", ns, name); @@ -679,21 +680,21 @@ private void wakeCluster(HiveCluster resource, KubernetesClient client) { // and use these values for spec.replicas. We don't call patchReplicas() // because the workloads may have been garbage-collected while suspended. int hs2Min = Math.max(1, spec.hiveServer2().autoscaling().minReplicas()); - HiveClusterAutoscaler.setManagedReplicas(ns, name, "hiveserver2", hs2Min); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_HIVESERVER2, hs2Min); if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { int hmsMin = Math.max(1, spec.metastore().autoscaling().minReplicas()); - HiveClusterAutoscaler.setManagedReplicas(ns, name, "metastore", hmsMin); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_METASTORE, hmsMin); } if (spec.llap().isEnabled()) { int llapWake = spec.llap().autoscaling().minReplicas(); - HiveClusterAutoscaler.setManagedReplicas(ns, name, "llap", llapWake); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_LLAP, llapWake); } if (spec.tezAm().isEnabled()) { int tezWake = spec.tezAm().autoscaling().minReplicas(); - HiveClusterAutoscaler.setManagedReplicas(ns, name, "tezam", tezWake); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_TEZAM, tezWake); } LOG.info("Cluster {}/{} woken up — restored to minReplicas", ns, name); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java index 111a496672da..c8ee85759678 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java @@ -46,6 +46,26 @@ */ public final class HiveWorkflowSpec implements WorkflowSpec { + // Dependent resource spec names (used as identifiers and dependency references) + private static final String HADOOP_CONFIGMAP = "hadoop-configmap"; + private static final String METASTORE_CONFIGMAP = "metastore-configmap"; + private static final String HIVESERVER2_CONFIGMAP = "hiveserver2-configmap"; + private static final String LLAP_CONFIGMAP = "llap-configmap"; + private static final String SCHEMA_INIT_JOB = "schema-init-job"; + private static final String METASTORE_DEPLOYMENT = "metastore-deployment"; + private static final String METASTORE_SERVICE = "metastore-service"; + private static final String HIVESERVER2_DEPLOYMENT = "hiveserver2-deployment"; + private static final String HIVESERVER2_SERVICE = "hiveserver2-service"; + private static final String LLAP_STATEFULSET = "llap-statefulset"; + private static final String LLAP_SERVICE = "llap-service"; + private static final String TEZAM_SERVICE = "tezam-service"; + private static final String TEZAM_STATEFULSET = "tezam-statefulset"; + private static final String SCRATCH_PVC = "scratch-pvc"; + private static final String HS2_PDB = "hs2-pdb"; + private static final String METASTORE_PDB = "metastore-pdb"; + private static final String LLAP_PDB = "llap-pdb"; + private static final String TEZAM_PDB = "tezam-pdb"; + private static final Condition METASTORE_ENABLED = (dr, primary, ctx) -> primary.getSpec().metastore().isEnabled(); @@ -80,93 +100,92 @@ private static List buildSpecs() { // --- ConfigMap dependents --- specs.add(new DependentResourceSpec( - HiveConfigMapDependent.Hadoop.class, "hadoop-configmap", + HiveConfigMapDependent.Hadoop.class, HADOOP_CONFIGMAP, Set.of(), null, null, null, null, null)); specs.add(new DependentResourceSpec( - HiveConfigMapDependent.Metastore.class, "metastore-configmap", + HiveConfigMapDependent.Metastore.class, METASTORE_CONFIGMAP, Set.of(), null, null, null, METASTORE_ENABLED, null)); specs.add(new DependentResourceSpec( - HiveConfigMapDependent.HiveServer2.class, "hiveserver2-configmap", + HiveConfigMapDependent.HiveServer2.class, HIVESERVER2_CONFIGMAP, Set.of(), null, null, null, null, null)); // --- Job dependents --- specs.add(new DependentResourceSpec( - SchemaInitJobDependent.class, "schema-init-job", - Set.of("metastore-configmap", "hadoop-configmap"), + SchemaInitJobDependent.class, SCHEMA_INIT_JOB, + Set.of(METASTORE_CONFIGMAP, HADOOP_CONFIGMAP), schemaJobCompleted(), null, null, METASTORE_ENABLED, null)); // --- Deployment dependents --- specs.add(new DependentResourceSpec( - MetastoreDeploymentDependent.class, "metastore-deployment", - Set.of("schema-init-job"), + MetastoreDeploymentDependent.class, METASTORE_DEPLOYMENT, + Set.of(SCHEMA_INIT_JOB), metastoreReady(), null, null, METASTORE_ENABLED, null)); // --- Service dependents --- specs.add(new DependentResourceSpec( - HiveServiceDependent.Metastore.class, "metastore-service", - Set.of("metastore-configmap"), + HiveServiceDependent.Metastore.class, METASTORE_SERVICE, + Set.of(METASTORE_CONFIGMAP), null, null, null, METASTORE_ENABLED, null)); specs.add(new DependentResourceSpec( - HiveServer2DeploymentDependent.class, "hiveserver2-deployment", - Set.of("hiveserver2-configmap", "hadoop-configmap"), + HiveServer2DeploymentDependent.class, HIVESERVER2_DEPLOYMENT, + Set.of(HIVESERVER2_CONFIGMAP, HADOOP_CONFIGMAP), null, hs2Precondition(), null, null, null)); specs.add(new DependentResourceSpec( - HiveServiceDependent.HiveServer2.class, "hiveserver2-service", - Set.of("hiveserver2-configmap"), + HiveServiceDependent.HiveServer2.class, HIVESERVER2_SERVICE, + Set.of(HIVESERVER2_CONFIGMAP), null, null, null, null, null)); // --- LLAP (conditional) --- specs.add(new DependentResourceSpec( - HiveConfigMapDependent.Llap.class, "llap-configmap", + HiveConfigMapDependent.Llap.class, LLAP_CONFIGMAP, Set.of(), null, null, null, LLAP_ENABLED, null)); specs.add(new DependentResourceSpec( - LlapStatefulSetDependent.class, "llap-statefulset", - Set.of("llap-configmap", "hadoop-configmap"), + LlapStatefulSetDependent.class, LLAP_STATEFULSET, + Set.of(LLAP_CONFIGMAP, HADOOP_CONFIGMAP), null, null, null, LLAP_ENABLED, null)); specs.add(new DependentResourceSpec( - HiveServiceDependent.Llap.class, "llap-service", + HiveServiceDependent.Llap.class, LLAP_SERVICE, Set.of(), null, null, null, LLAP_ENABLED, null)); // --- TezAM (conditional) --- specs.add(new DependentResourceSpec( - ScratchPvcDependent.class, "scratch-pvc", + ScratchPvcDependent.class, SCRATCH_PVC, Set.of(), null, null, null, TEZAM_ENABLED, null)); specs.add(new DependentResourceSpec( - HiveServiceDependent.TezAm.class, "tezam-service", + HiveServiceDependent.TezAm.class, TEZAM_SERVICE, Set.of(), null, null, null, TEZAM_ENABLED, null)); specs.add(new DependentResourceSpec( - TezAmStatefulSetDependent.class, "tezam-statefulset", - Set.of("hiveserver2-configmap", "hadoop-configmap", "tezam-service", "scratch-pvc"), + TezAmStatefulSetDependent.class, TEZAM_STATEFULSET, + Set.of(HIVESERVER2_CONFIGMAP, HADOOP_CONFIGMAP, TEZAM_SERVICE, SCRATCH_PVC), null, null, null, TEZAM_ENABLED, null)); - // --- Autoscaling: PodDisruptionBudgets (conditional) --- specs.add(new DependentResourceSpec( - HivePdbDependent.HiveServer2.class, "hs2-pdb", - Set.of("hiveserver2-deployment"), + HivePdbDependent.HiveServer2.class, HS2_PDB, + Set.of(HIVESERVER2_DEPLOYMENT), null, HS2_AUTOSCALING, null, null, null)); specs.add(new DependentResourceSpec( - HivePdbDependent.Metastore.class, "metastore-pdb", - Set.of("metastore-deployment"), + HivePdbDependent.Metastore.class, METASTORE_PDB, + Set.of(METASTORE_DEPLOYMENT), null, METASTORE_AUTOSCALING, null, null, null)); specs.add(new DependentResourceSpec( - HivePdbDependent.Llap.class, "llap-pdb", - Set.of("llap-statefulset"), + HivePdbDependent.Llap.class, LLAP_PDB, + Set.of(LLAP_STATEFULSET), null, LLAP_AUTOSCALING, null, null, null)); specs.add(new DependentResourceSpec( - HivePdbDependent.TezAm.class, "tezam-pdb", - Set.of("tezam-statefulset"), + HivePdbDependent.TezAm.class, TEZAM_PDB, + Set.of(TEZAM_STATEFULSET), null, TEZAM_AUTOSCALING, null, null, null)); return Collections.unmodifiableList(specs); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 0415865f1c9d..1b37f66a5844 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -25,6 +25,12 @@ public final class ConfigUtils { private ConfigUtils() { } + // Component names used across the operator for labels, resource naming, and autoscaling keys. + public static final String COMPONENT_HIVESERVER2 = "hiveserver2"; + public static final String COMPONENT_METASTORE = "metastore"; + public static final String COMPONENT_LLAP = "llap"; + public static final String COMPONENT_TEZAM = "tezam"; + public static final String METASTORE_THRIFT_PORT_KEY = "metastore.thrift.port"; public static final String METASTORE_THRIFT_PORT_HIVE_KEY = "hive.metastore.port"; public static final int METASTORE_THRIFT_PORT_DEFAULT = 9083; From 67525541b34e47abd0baf5d363ad5837d60ddb87 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 10 Jun 2026 04:54:07 +0530 Subject: [PATCH 13/22] Make port configurable --- .../crds/hiveclusters.hive.apache.org-v1.yml | 24 +++++++++++++++++++ .../autoscaling/HiveClusterAutoscaler.java | 10 ++++---- .../operator/autoscaling/MetricsScraper.java | 10 ++++---- .../dependent/HiveDependentResource.java | 6 ++--- .../HiveServer2DeploymentDependent.java | 2 +- .../dependent/LlapStatefulSetDependent.java | 2 +- .../MetastoreDeploymentDependent.java | 2 +- .../operator/model/spec/AutoscalingSpec.java | 8 ++++++- .../operator/model/spec/HiveServer2Spec.java | 2 +- .../operator/model/spec/LlapSpec.java | 2 +- .../operator/model/spec/MetastoreSpec.java | 2 +- .../operator/model/spec/TezAmSpec.java | 2 +- 12 files changed, 52 insertions(+), 20 deletions(-) diff --git a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml index 913692c99fff..fdd8ef33da25 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -105,6 +105,12 @@ spec: The pod terminates immediately once sessions/connections drain to 0; this value is only the upper safety cap. type: integer + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer metricsScrapeIntervalSeconds: default: 10 description: How often (seconds) the operator scrapes JMX @@ -271,6 +277,12 @@ spec: The pod terminates immediately once sessions/connections drain to 0; this value is only the upper safety cap. type: integer + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer metricsScrapeIntervalSeconds: default: 10 description: How often (seconds) the operator scrapes JMX @@ -412,6 +424,12 @@ spec: The pod terminates immediately once sessions/connections drain to 0; this value is only the upper safety cap. type: integer + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer metricsScrapeIntervalSeconds: default: 10 description: How often (seconds) the operator scrapes JMX @@ -610,6 +628,12 @@ spec: The pod terminates immediately once sessions/connections drain to 0; this value is only the upper safety cap. type: integer + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer metricsScrapeIntervalSeconds: default: 10 description: How often (seconds) the operator scrapes JMX diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index e345102482d1..64061eec8fb6 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -103,11 +103,12 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie // HiveServer2 if (spec.hiveServer2().autoscaling().isEnabled()) { + AutoscalingSpec hs2Auto = spec.hiveServer2().autoscaling(); Map hs2Selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); - List hs2Metrics = scraper.scrape(namespace, hs2Selector); + List hs2Metrics = scraper.scrape(namespace, hs2Selector, hs2Auto.metricsPort()); updatePodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); evaluateComponent(cluster, client, namespace, clusterName, - ConfigUtils.COMPONENT_HIVESERVER2, spec.hiveServer2().autoscaling(), + ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, spec.hiveServer2().replicas(), patches, statuses, hs2Metrics); } @@ -141,7 +142,8 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie public List scrapeHs2Metrics(HiveCluster cluster) { String namespace = cluster.getMetadata().getNamespace(); Map selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); - return scraper.scrape(namespace, selector); + int port = cluster.getSpec().hiveServer2().autoscaling().metricsPort(); + return scraper.scrape(namespace, selector, port); } private void evaluateComponent(HiveCluster cluster, KubernetesClient client, @@ -167,7 +169,7 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, metrics = preScrapedMetrics; } else { Map selector = Labels.selectorForComponent(cluster, component); - metrics = scraper.scrape(namespace, selector); + metrics = scraper.scrape(namespace, selector, autoscaling.metricsPort()); } // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java index c30702d8c639..c3aa8aa82ca8 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java @@ -42,7 +42,6 @@ public class MetricsScraper { private static final Logger LOG = LoggerFactory.getLogger(MetricsScraper.class); - private static final int JMX_EXPORTER_PORT = 9404; private static final Duration TIMEOUT = Duration.ofSeconds(5); private final KubernetesClient client; @@ -62,9 +61,10 @@ public MetricsScraper(KubernetesClient client) { * * @param namespace the namespace to query * @param selector label selector (e.g., app.kubernetes.io/component=hiveserver2) + * @param metricsPort the port on which the Prometheus JMX Exporter serves metrics * @return list of per-pod metrics (empty if no pods or all fail) */ - public List scrape(String namespace, Map selector) { + public List scrape(String namespace, Map selector, int metricsPort) { List pods; try { pods = client.pods() @@ -95,7 +95,7 @@ public List scrape(String namespace, Map selector) { for (Pod pod : scrapeable) { String podName = pod.getMetadata().getName(); String podIp = pod.getStatus().getPodIP(); - futures.add(fetchMetricsAsync(podIp) + futures.add(fetchMetricsAsync(podIp, metricsPort) .thenApply(body -> new PodMetrics(podName, PrometheusTextParser.parse(body))) .exceptionally(ex -> { LOG.debug("Failed to scrape metrics from pod {}: {}", podName, ex.getMessage()); @@ -116,8 +116,8 @@ public List scrape(String namespace, Map selector) { return results; } - private CompletableFuture fetchMetricsAsync(String podIp) { - URI uri = URI.create("http://" + podIp + ":" + JMX_EXPORTER_PORT + "/metrics"); + private CompletableFuture fetchMetricsAsync(String podIp, int metricsPort) { + URI uri = URI.create("http://" + podIp + ":" + metricsPort + "/metrics"); HttpRequest request = HttpRequest.newBuilder() .uri(uri) .timeout(TIMEOUT) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 1728dbb33f91..64370cd412db 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -742,7 +742,7 @@ protected static void appendUserVolumes( * @param ports list to add the metrics port to */ protected static void addJmxExporter( - String image, String component, + String image, String component, int metricsPort, List initContainers, List volumeMounts, List volumes, @@ -778,13 +778,13 @@ protected static void addJmxExporter( // Expose the metrics port ports.add(new io.fabric8.kubernetes.api.model.ContainerPortBuilder() .withName("metrics") - .withContainerPort(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT) + .withContainerPort(metricsPort) .withProtocol("TCP").build()); // Add javaagent flag to the appropriate JVM opts env var. // LLAP uses LLAP_DAEMON_OPTS (its startup script ignores SERVICE_OPTS). String agentArg = String.format("-javaagent:%s=%d:%s", - JMX_EXPORTER_JAR, ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT, JMX_EXPORTER_CONFIG); + JMX_EXPORTER_JAR, metricsPort, JMX_EXPORTER_CONFIG); String optsEnvVar = ConfigUtils.COMPONENT_LLAP.equals(component) ? "LLAP_DAEMON_OPTS" : "SERVICE_OPTS"; boolean found = false; for (int i = 0; i < envVars.size(); i++) { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index d7d81a3ce2a1..bf18ed7a7236 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -207,7 +207,7 @@ protected Deployment desired(HiveCluster hiveCluster, // Add Prometheus JMX Exporter when autoscaling is enabled AutoscalingSpec autoscaling = hs2.autoscaling(); if (autoscaling.isEnabled()) { - addJmxExporter(spec.image(), COMPONENT, + addJmxExporter(spec.image(), COMPONENT, autoscaling.metricsPort(), initContainers, volumeMounts, volumes, envVars, ports); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index dad4ee6500d6..e4c8cac0b5e1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -132,7 +132,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, // Add Prometheus JMX Exporter when autoscaling is enabled AutoscalingSpec autoscaling = llap.autoscaling(); if (autoscaling.isEnabled()) { - addJmxExporter(spec.image(), COMPONENT, + addJmxExporter(spec.image(), COMPONENT, autoscaling.metricsPort(), initContainers, volumeMounts, volumes, envVars, ports); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index 9784d1df0c76..bde9a5edf68f 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -121,7 +121,7 @@ protected Deployment desired(HiveCluster hiveCluster, // Add Prometheus JMX Exporter when autoscaling is enabled AutoscalingSpec autoscaling = spec.metastore().autoscaling(); if (autoscaling.isEnabled()) { - addJmxExporter(spec.image(), COMPONENT, + addJmxExporter(spec.image(), COMPONENT, autoscaling.metricsPort(), initContainers, volumeMounts, volumes, envVars, ports); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java index 9f0a0a908d7c..fac4b016766a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonPropertyDescription; import io.fabric8.generator.annotation.Default; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; /** Autoscaling configuration for a Hive component. The operator scrapes JMX metrics directly from pods. */ public record AutoscalingSpec( @@ -61,7 +62,11 @@ public record AutoscalingSpec( @JsonPropertyDescription("CPU percentage (0-100) below which scale-down is considered. " + "Only applies to HS2 and HMS.") @Default("30") - Integer cpuScaleDownThreshold) { + Integer cpuScaleDownThreshold, + @JsonPropertyDescription("Port on which the Prometheus JMX Exporter serves metrics. " + + "The operator scrapes this port on each pod for autoscaling decisions.") + @Default("9404") + Integer metricsPort) { public AutoscalingSpec { enabled = enabled != null ? enabled : false; @@ -73,6 +78,7 @@ public record AutoscalingSpec( metricsScrapeIntervalSeconds = metricsScrapeIntervalSeconds != null ? metricsScrapeIntervalSeconds : 10; cpuScaleUpThreshold = cpuScaleUpThreshold != null ? cpuScaleUpThreshold : 90; cpuScaleDownThreshold = cpuScaleDownThreshold != null ? cpuScaleDownThreshold : 30; + metricsPort = metricsPort != null ? metricsPort : ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT; } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java index fc934d955ddb..5f00a5c1dde5 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -62,6 +62,6 @@ public record HiveServer2Spec( extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); externalJars = externalJars != null ? externalJars : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 80, 60, 600, 300, 10, 90, 30); + false, 1, 80, 60, 600, 300, 10, 90, 30, null); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index c058232032a6..b4865d66ce43 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -68,7 +68,7 @@ public record LlapSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 1, 60, 900, 600, 10, 0, 0); + false, 0, 1, 60, 900, 600, 10, 0, 0, null); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 5957eb989afc..e1a0ac1452ae 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -69,7 +69,7 @@ public record MetastoreSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 1, 75, 60, 300, 60, 10, 90, 30); + false, 1, 75, 60, 300, 60, 10, 90, 30, null); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index f11fd6e417d0..606b0de14053 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -63,7 +63,7 @@ public record TezAmSpec( extraVolumes = extraVolumes != null ? extraVolumes : List.of(); extraVolumeMounts = extraVolumeMounts != null ? extraVolumeMounts : List.of(); autoscaling = autoscaling != null ? autoscaling : new AutoscalingSpec( - false, 0, 0, 60, 600, 120, 10, 0, 0); + false, 0, 0, 60, 600, 120, 10, 0, 0, null); } public boolean isEnabled() { From 8af73b0674706724727706bc0857b91d756090be Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 10 Jun 2026 07:30:49 +0530 Subject: [PATCH 14/22] Formatting issues --- .../autoscaling/HiveClusterAutoscaler.java | 10 +- .../dependent/HiveDependentResource.java | 116 +++++++------- .../reconciler/HiveClusterReconciler.java | 141 +++++++++--------- 3 files changed, 133 insertions(+), 134 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 64061eec8fb6..3aa2f831c8b4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -222,11 +222,11 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, private ScalingStrategy createStrategy(String component, HiveCluster cluster) { return switch (component) { - case ConfigUtils.COMPONENT_HIVESERVER2 -> new HiveServer2ScalingStrategy(); - case ConfigUtils.COMPONENT_METASTORE -> new MetastoreScalingStrategy(); - case ConfigUtils.COMPONENT_LLAP -> new LlapScalingStrategy(this, cluster); - case ConfigUtils.COMPONENT_TEZAM -> new TezAmScalingStrategy(this, cluster); - default -> throw new IllegalArgumentException("Unknown component: " + component); + case ConfigUtils.COMPONENT_HIVESERVER2 -> new HiveServer2ScalingStrategy(); + case ConfigUtils.COMPONENT_METASTORE -> new MetastoreScalingStrategy(); + case ConfigUtils.COMPONENT_LLAP -> new LlapScalingStrategy(this, cluster); + case ConfigUtils.COMPONENT_TEZAM -> new TezAmScalingStrategy(this, cluster); + default -> throw new IllegalArgumentException("Unknown component: " + component); }; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 64370cd412db..fd8458fe46ec 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -812,64 +812,64 @@ private static String buildJmxExporterConfig(String component) { sb.append("rules:\n"); switch (component) { - case ConfigUtils.COMPONENT_HIVESERVER2: - // HS2 session and operation metrics - sb.append("- pattern: 'metrics<>Value'\n"); - sb.append(" name: hs2_$1\n"); - sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'metrics<>Count'\n"); - sb.append(" name: hs2_active_calls_$1\n"); - sb.append(" type: GAUGE\n"); - // Tez session pool metrics (pending tasks, backlog ratio, running tasks) - sb.append("- pattern: 'metrics<>Value'\n"); - sb.append(" name: tez_session_$1\n"); - sb.append(" type: GAUGE\n"); - // JVM CPU usage for CPU-based autoscaling - sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); - sb.append(" name: jvm_process_cpu_load\n"); - sb.append(" type: GAUGE\n"); - break; - case ConfigUtils.COMPONENT_METASTORE: - // HMS API call metrics - sb.append("- pattern: 'metrics<>Count'\n"); - sb.append(" name: api_$1_total\n"); - sb.append(" type: COUNTER\n"); - sb.append("- pattern: 'metrics<>Count'\n"); - sb.append(" name: hive_metastore_open_connections\n"); - sb.append(" type: GAUGE\n"); - // JVM CPU usage for CPU-based autoscaling - sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); - sb.append(" name: jvm_process_cpu_load\n"); - sb.append(" type: GAUGE\n"); - break; - case ConfigUtils.COMPONENT_LLAP: - // Only export the executor metrics the autoscaler and drain script need. - // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, - // causing CPU spikes and GC pressure on the LLAP JVM. - // Internal format: Hadoop><>Attribute - // Separate rules per attribute — JMX Exporter 1.x caches per-bean, not per-attribute. - sb.append("- pattern: 'Hadoop<>ExecutorNumQueuedRequests'\n"); - sb.append(" name: hadoop_llapdaemon_executornumqueuedrequests\n"); - sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsConfigured'\n"); - sb.append(" name: hadoop_llapdaemon_executornumexecutorsconfigured\n"); - sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsAvailable'\n"); - sb.append(" name: hadoop_llapdaemon_executornumexecutorsavailable\n"); - sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutors'\n"); - sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); - sb.append(" type: GAUGE\n"); - break; - case ConfigUtils.COMPONENT_TEZAM: - // TezAM DAG execution metrics - sb.append("- pattern: 'Hadoop<>(.+)'\n"); - sb.append(" name: tez_am_$1\n"); - sb.append(" type: GAUGE\n"); - break; - default: - sb.append("- pattern: '.*'\n"); - break; + case ConfigUtils.COMPONENT_HIVESERVER2: + // HS2 session and operation metrics + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: hs2_$1\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: hs2_active_calls_$1\n"); + sb.append(" type: GAUGE\n"); + // Tez session pool metrics (pending tasks, backlog ratio, running tasks) + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: tez_session_$1\n"); + sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_METASTORE: + // HMS API call metrics + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: api_$1_total\n"); + sb.append(" type: COUNTER\n"); + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: hive_metastore_open_connections\n"); + sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_LLAP: + // Only export the executor metrics the autoscaler and drain script need. + // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, + // causing CPU spikes and GC pressure on the LLAP JVM. + // Internal format: Hadoop><>Attribute + // Separate rules per attribute — JMX Exporter 1.x caches per-bean, not per-attribute. + sb.append("- pattern: 'Hadoop<>ExecutorNumQueuedRequests'\n"); + sb.append(" name: hadoop_llapdaemon_executornumqueuedrequests\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsConfigured'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsconfigured\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsAvailable'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsavailable\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutors'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_TEZAM: + // TezAM DAG execution metrics + sb.append("- pattern: 'Hadoop<>(.+)'\n"); + sb.append(" name: tez_am_$1\n"); + sb.append(" type: GAUGE\n"); + break; + default: + sb.append("- pattern: '.*'\n"); + break; } return sb.toString(); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 38b69bdd42fe..31bead97718d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -77,72 +77,72 @@ public UpdateControl reconcile(HiveCluster resource, Context Date: Wed, 10 Jun 2026 12:55:10 +0530 Subject: [PATCH 15/22] Remove Hardcoded Ports --- .../dependent/HiveDependentResource.java | 12 --------- .../dependent/HiveServiceDependent.java | 25 ++++++++++++++----- .../dependent/LlapStatefulSetDependent.java | 23 +++++++++++++---- .../MetastoreDeploymentDependent.java | 6 ++++- .../kubernetes/operator/util/ConfigUtils.java | 18 +++++++++++++ 5 files changed, 60 insertions(+), 24 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index fd8458fe46ec..9f7a345daffd 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -106,10 +106,6 @@ public Optional getSecondaryResource(P primary, if (resources.isEmpty()) { return Optional.empty(); } - // Always filter by expected name — even when only one resource - // is in the cache. Without this, a single Deployment (e.g. - // metastore) would be handed to HiveServer2's matcher, causing - // a cross-component update loop. String expectedName = getSecondaryResourceName(primary, context); return resources.stream() @@ -138,14 +134,6 @@ public Matcher.Result match(R actualResource, R desired, return super.match(actualResource, desired, primary, context); } - /** - * Handles 409 Conflict errors during resource creation caused by informer - * cache lag. When the operator creates a resource but the informer hasn't - * yet received the creation event, the framework may attempt to create it - * again. Kubernetes rejects the duplicate with 409 — this handler absorbs - * that expected race and lets the next reconciliation pick up the resource - * from the updated cache. - */ @Override protected R handleCreate(R desired, P primary, Context

context) { try { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java index 55fe15e182d2..e367bf4c6028 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -114,17 +114,20 @@ public Metastore() { @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { - int thriftPort = ConfigUtils.getInt( - hiveCluster.getSpec().metastore().configOverrides(), + var overrides = hiveCluster.getSpec().metastore().configOverrides(); + int thriftPort = ConfigUtils.getInt(overrides, ConfigUtils.METASTORE_THRIFT_PORT_KEY, ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); + int restPort = ConfigUtils.getInt(overrides, + ConfigUtils.METASTORE_REST_HTTP_PORT_KEY, + null, ConfigUtils.METASTORE_REST_HTTP_PORT_DEFAULT); builder.editSpec() .withType("ClusterIP") .addNewPort().withName("thrift").withProtocol("TCP") .withPort(thriftPort).withTargetPort(new IntOrString(thriftPort)).endPort() .addNewPort().withName("rest").withProtocol("TCP") - .withPort(9001).withTargetPort(new IntOrString(9001)).endPort() + .withPort(restPort).withTargetPort(new IntOrString(restPort)).endPort() .endSpec(); } } @@ -141,14 +144,24 @@ public Llap() { @Override protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + var overrides = hiveCluster.getSpec().llap().configOverrides(); + int managementPort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT); + int shufflePort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT); + int webPort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT); builder.editSpec() .withClusterIP("None") .addNewPort().withName("management").withProtocol("TCP") - .withPort(15004).withTargetPort(new IntOrString(15004)).endPort() + .withPort(managementPort).withTargetPort(new IntOrString(managementPort)).endPort() .addNewPort().withName("shuffle").withProtocol("TCP") - .withPort(15551).withTargetPort(new IntOrString(15551)).endPort() + .withPort(shufflePort).withTargetPort(new IntOrString(shufflePort)).endPort() .addNewPort().withName("web").withProtocol("TCP") - .withPort(15002).withTargetPort(new IntOrString(15002)).endPort() + .withPort(webPort).withTargetPort(new IntOrString(webPort)).endPort() .endSpec(); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index e4c8cac0b5e1..380a7bea7967 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -96,17 +96,30 @@ protected StatefulSet desired(HiveCluster hiveCluster, envVars.addAll(spec.envVars()); } + int managementPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT); + int shufflePort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT); + int webPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT); + int outputPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_DEFAULT); + List ports = new ArrayList<>(); ports.add(new ContainerPortBuilder() - .withName("management").withContainerPort(15004).withProtocol("TCP").build()); + .withName("management").withContainerPort(managementPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("shuffle").withContainerPort(15551).withProtocol("TCP").build()); + .withName("shuffle").withContainerPort(shufflePort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("web").withContainerPort(15002).withProtocol("TCP").build()); + .withName("web").withContainerPort(webPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("output").withContainerPort(15003).withProtocol("TCP").build()); + .withName("output").withContainerPort(outputPort).withProtocol("TCP").build()); - Probe readinessProbe = buildTcpProbe(15004, llap.readinessProbe(), 15, 10, 3); + Probe readinessProbe = buildTcpProbe(managementPort, llap.readinessProbe(), 15, 10, 3); String headlessServiceName = hiveCluster.getMetadata().getName() + "-llap"; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java index bde9a5edf68f..ff19afd5c023 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -89,11 +89,15 @@ protected Deployment desired(HiveCluster hiveCluster, ConfigUtils.METASTORE_THRIFT_PORT_KEY, ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); + int restPort = ConfigUtils.getInt( + spec.metastore().configOverrides(), + ConfigUtils.METASTORE_REST_HTTP_PORT_KEY, + null, ConfigUtils.METASTORE_REST_HTTP_PORT_DEFAULT); List ports = new ArrayList<>(); ports.add(new ContainerPortBuilder() .withName("thrift").withContainerPort(thriftPort).withProtocol("TCP").build()); ports.add(new ContainerPortBuilder() - .withName("rest").withContainerPort(9001).withProtocol("TCP").build()); + .withName("rest").withContainerPort(restPort).withProtocol("TCP").build()); Probe readinessProbe = buildTcpProbe(thriftPort, spec.metastore().readinessProbe(), 15, 10, 3); Probe livenessProbe = buildTcpProbe(thriftPort, spec.metastore().livenessProbe(), 60, 30, 5); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 1b37f66a5844..8761b23b298a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -79,6 +79,21 @@ private ConfigUtils() { public static final String HIVE_LLAP_DAEMON_NUM_EXECUTORS_KEY = "hive.llap.daemon.num.executors"; + public static final String HIVE_LLAP_DAEMON_RPC_PORT_KEY = "hive.llap.daemon.rpc.port"; + public static final int HIVE_LLAP_DAEMON_RPC_PORT_DEFAULT = 15001; + + public static final String HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY = "hive.llap.management.rpc.port"; + public static final int HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT = 15004; + + public static final String HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY = "hive.llap.daemon.yarn.shuffle.port"; + public static final int HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT = 15551; + + public static final String HIVE_LLAP_DAEMON_WEB_PORT_KEY = "hive.llap.daemon.web.port"; + public static final int HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT = 15002; + + public static final String HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_KEY = "hive.llap.daemon.output.service.port"; + public static final int HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_DEFAULT = 15003; + public static final String METASTORE_SERVER_TRANSPORT_MODE_KEY = "metastore.server.thrift.transport.mode"; public static final String METASTORE_SERVER_TRANSPORT_MODE_DEFAULT = "http"; @@ -95,6 +110,9 @@ private ConfigUtils() { public static final String METASTORE_SERVER_MAX_THREADS_HIVE_KEY = "hive.metastore.server.max.threads"; public static final int METASTORE_SERVER_MAX_THREADS_DEFAULT = 1000; + public static final String METASTORE_REST_HTTP_PORT_KEY = "metastore.rest.http.port"; + public static final int METASTORE_REST_HTTP_PORT_DEFAULT = 9001; + public static final String HIVE_METASTORE_URIS_KEY = "hive.metastore.uris"; public static final String HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY = "hive.server2.tez.sessions.per.default.queue"; From 38356af16b609736782343d3dc60d22dbd0bca6d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 10 Jun 2026 13:29:06 +0530 Subject: [PATCH 16/22] Refactor --- .../autoscaling/BackgroundMetricsScraper.java | 132 +++++++++++++++ .../autoscaling/HiveClusterAutoscaler.java | 158 ++++++++++++++---- .../autoscaling/LlapScalingStrategy.java | 2 +- .../operator/autoscaling/MetricsCache.java | 72 ++++++++ .../autoscaling/TezAmScalingStrategy.java | 2 +- .../operator/dependent/HivePdbDependent.java | 6 +- .../reconciler/HiveClusterReconciler.java | 33 +++- 7 files changed, 359 insertions(+), 46 deletions(-) create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java create mode 100644 packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java new file mode 100644 index 000000000000..26c2fc83d668 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java @@ -0,0 +1,132 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Runs periodic metrics scraping in the background so that the JOSDK reconcile + * thread is never blocked by HTTP calls to pod JMX exporters. + *

+ * Each component gets its own scheduled task that writes results to a shared + * {@link MetricsCache}. The reconciler reads from that cache (non-blocking). + */ +public class BackgroundMetricsScraper { + + private static final Logger LOG = LoggerFactory.getLogger(BackgroundMetricsScraper.class); + + private final ScheduledExecutorService scheduler; + private final MetricsScraper scraper; + private final MetricsCache cache; + // Key: "namespace/clusterName/component" → active scrape task + private final ConcurrentHashMap> activeTasks = + new ConcurrentHashMap<>(); + // Tracks registered intervals to detect spec changes + private final ConcurrentHashMap registeredIntervals = + new ConcurrentHashMap<>(); + + public BackgroundMetricsScraper(MetricsScraper scraper, MetricsCache cache) { + this.scraper = scraper; + this.cache = cache; + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = new Thread(r, "hive-metrics-scraper"); + t.setDaemon(true); + return t; + }); + } + + /** + * Registers (or updates) a periodic scrape task for a component. + * Idempotent — only recreates the task if the interval has changed. + * + * @param namespace the Kubernetes namespace + * @param clusterName the HiveCluster name + * @param component the component name (e.g., "hiveserver2") + * @param selector label selector for pod listing + * @param metricsPort the JMX exporter port + * @param intervalSecs how often to scrape (from AutoscalingSpec) + */ + public void registerOrUpdate(String namespace, String clusterName, + String component, Map selector, + int metricsPort, int intervalSecs) { + String key = namespace + "/" + clusterName + "/" + component; + Integer existing = registeredIntervals.get(key); + if (existing != null && existing == intervalSecs) { + return; // Already registered with same interval + } + + // Cancel existing task if interval changed + ScheduledFuture oldTask = activeTasks.remove(key); + if (oldTask != null) { + oldTask.cancel(false); + } + + ScheduledFuture future = scheduler.scheduleWithFixedDelay( + () -> scrapeAndStore(key, namespace, selector, metricsPort), + 0, intervalSecs, TimeUnit.SECONDS); + + activeTasks.put(key, future); + registeredIntervals.put(key, intervalSecs); + LOG.debug("Registered background scrape for {} (interval={}s)", key, intervalSecs); + } + + /** + * Unregisters all scrape tasks for a deleted cluster. + */ + public void unregisterCluster(String namespace, String clusterName) { + String prefix = namespace + "/" + clusterName + "/"; + activeTasks.entrySet().removeIf(entry -> { + if (entry.getKey().startsWith(prefix)) { + entry.getValue().cancel(false); + return true; + } + return false; + }); + registeredIntervals.keySet().removeIf(k -> k.startsWith(prefix)); + cache.removeByPrefix(prefix); + LOG.debug("Unregistered background scrape tasks for {}/{}", namespace, clusterName); + } + + /** + * Shuts down the background scheduler. Called on operator shutdown. + */ + public void shutdown() { + scheduler.shutdownNow(); + } + + private void scrapeAndStore(String key, String namespace, + Map selector, int metricsPort) { + try { + List metrics = scraper.scrape(namespace, selector, metricsPort); + cache.put(key, metrics); + } catch (Exception e) { + // Do not update cache on failure — staleness check handles it + LOG.debug("Background scrape failed for {}: {}", key, e.getMessage()); + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 3aa2f831c8b4..b091f397c485 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -18,6 +18,7 @@ package org.apache.hive.kubernetes.operator.autoscaling; +import java.time.Duration; import java.time.Instant; import java.util.HashMap; import java.util.List; @@ -72,19 +73,51 @@ public static void setManagedReplicas(String namespace, String clusterName, MANAGED_REPLICAS.put(namespace + "/" + clusterName + "/" + component, replicas); } + private record PendingScaleDown(int targetReplicas, Instant annotatedAt) {} + private final MetricsScraper scraper; + private final BackgroundMetricsScraper bgScraper; + private final MetricsCache metricsCache; // Key: "namespace/clusterName/component" private final ConcurrentHashMap autoscalers = new ConcurrentHashMap<>(); private final ConcurrentHashMap lastScaleTimes = new ConcurrentHashMap<>(); + // Two-phase scale-down: holds deferred scale-down targets while pod-deletion-cost + // annotations propagate (2s delay before applying the actual scale patch). + private final ConcurrentHashMap pendingScaleDowns = + new ConcurrentHashMap<>(); - public HiveClusterAutoscaler(MetricsScraper scraper) { + public HiveClusterAutoscaler(MetricsScraper scraper, + BackgroundMetricsScraper bgScraper, MetricsCache metricsCache) { this.scraper = scraper; + this.bgScraper = bgScraper; + this.metricsCache = metricsCache; + } + + public BackgroundMetricsScraper getBackgroundScraper() { + return bgScraper; } - public MetricsScraper getScraper() { - return scraper; + /** + * Removes all in-memory state for a deleted HiveCluster to prevent memory leaks. + */ + public void cleanupCluster(String namespace, String clusterName) { + String prefix = namespace + "/" + clusterName + "/"; + MANAGED_REPLICAS.keySet().removeIf(k -> k.startsWith(prefix)); + autoscalers.keySet().removeIf(k -> k.startsWith(prefix)); + lastScaleTimes.keySet().removeIf(k -> k.startsWith(prefix)); + pendingScaleDowns.keySet().removeIf(k -> k.startsWith(prefix)); + LOG.info("Cleaned up autoscaler state for {}/{}", namespace, clusterName); + } + + /** + * Returns true if there are pending scale-down operations waiting for + * annotation propagation. The reconciler should reschedule sooner (2s) + * when this returns true. + */ + public boolean hasPendingScaleDowns() { + return !pendingScaleDowns.isEmpty(); } /** @@ -104,74 +137,122 @@ public AutoscalingEvaluation evaluate(HiveCluster cluster, KubernetesClient clie // HiveServer2 if (spec.hiveServer2().autoscaling().isEnabled()) { AutoscalingSpec hs2Auto = spec.hiveServer2().autoscaling(); + String hs2Key = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_HIVESERVER2; Map hs2Selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); - List hs2Metrics = scraper.scrape(namespace, hs2Selector, hs2Auto.metricsPort()); - updatePodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); - evaluateComponent(cluster, client, namespace, clusterName, - ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, - spec.hiveServer2().replicas(), patches, statuses, hs2Metrics); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Selector, + hs2Auto.metricsPort(), hs2Auto.metricsScrapeIntervalSeconds()); + int maxStale = hs2Auto.metricsScrapeIntervalSeconds() * 3; + List hs2Metrics = metricsCache.getOrEmpty(hs2Key, maxStale); + + // Two-phase scale-down: check if a pending scale-down from a prior + // reconcile is ready to be applied (annotations have propagated). + PendingScaleDown pending = pendingScaleDowns.get(hs2Key); + if (pending != null) { + if (Duration.between(pending.annotatedAt(), Instant.now()).toSeconds() >= 2) { + patches.put(ConfigUtils.COMPONENT_HIVESERVER2, pending.targetReplicas()); + MANAGED_REPLICAS.put(hs2Key, pending.targetReplicas()); + lastScaleTimes.put(hs2Key, Instant.now().toString()); + pendingScaleDowns.remove(hs2Key); + LOG.info("[hiveserver2] Applying deferred scale-down to {} replicas", pending.targetReplicas()); + } + // Build status even when waiting for pending scale-down + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, + spec.hiveServer2().replicas(), new HashMap<>(), statuses, hs2Metrics); + } else { + // Pod deletion cost only applies to Deployments (ReplicaSet controller). + // StatefulSets always scale down by highest ordinal regardless of this + // annotation. LLAP/TezAM graceful drain is handled by preStop hooks. + updateDeploymentPodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); + + Map hs2Patches = new HashMap<>(); + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, + spec.hiveServer2().replicas(), hs2Patches, statuses, hs2Metrics); + + Integer hs2Patch = hs2Patches.get(ConfigUtils.COMPONENT_HIVESERVER2); + int currentReplicas = getCurrentReplicas(client, namespace, clusterName, ConfigUtils.COMPONENT_HIVESERVER2); + if (hs2Patch != null && hs2Patch < currentReplicas) { + // Scale-down: defer to allow deletion-cost annotations to propagate + pendingScaleDowns.put(hs2Key, new PendingScaleDown(hs2Patch, Instant.now())); + LOG.info("[hiveserver2] Deferring scale-down to {} (waiting for deletion-cost propagation)", + hs2Patch); + } else if (hs2Patch != null) { + // Scale-up: apply immediately + patches.put(ConfigUtils.COMPONENT_HIVESERVER2, hs2Patch); + MANAGED_REPLICAS.put(hs2Key, hs2Patch); + } + } } // Metastore if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { + AutoscalingSpec msAuto = spec.metastore().autoscaling(); + Map msSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_METASTORE); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_METASTORE, msSelector, + msAuto.metricsPort(), msAuto.metricsScrapeIntervalSeconds()); + String msKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_METASTORE; + List msMetrics = metricsCache.getOrEmpty(msKey, msAuto.metricsScrapeIntervalSeconds() * 3); evaluateComponent(cluster, client, namespace, clusterName, - ConfigUtils.COMPONENT_METASTORE, spec.metastore().autoscaling(), - spec.metastore().replicas(), patches, statuses); + ConfigUtils.COMPONENT_METASTORE, msAuto, + spec.metastore().replicas(), patches, statuses, msMetrics); } // LLAP if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { + AutoscalingSpec llapAuto = spec.llap().autoscaling(); + Map llapSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_LLAP); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_LLAP, llapSelector, + llapAuto.metricsPort(), llapAuto.metricsScrapeIntervalSeconds()); + String llapKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_LLAP; + List llapMetrics = metricsCache.getOrEmpty(llapKey, llapAuto.metricsScrapeIntervalSeconds() * 3); evaluateComponent(cluster, client, namespace, clusterName, - ConfigUtils.COMPONENT_LLAP, spec.llap().autoscaling(), - spec.llap().replicas(), patches, statuses); + ConfigUtils.COMPONENT_LLAP, llapAuto, + spec.llap().replicas(), patches, statuses, llapMetrics); } // TezAM if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { + AutoscalingSpec tezAuto = spec.tezAm().autoscaling(); + Map tezSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_TEZAM); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_TEZAM, tezSelector, + tezAuto.metricsPort(), tezAuto.metricsScrapeIntervalSeconds()); + String tezKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_TEZAM; + List tezMetrics = metricsCache.getOrEmpty(tezKey, tezAuto.metricsScrapeIntervalSeconds() * 3); evaluateComponent(cluster, client, namespace, clusterName, - ConfigUtils.COMPONENT_TEZAM, spec.tezAm().autoscaling(), - spec.tezAm().replicas(), patches, statuses); + ConfigUtils.COMPONENT_TEZAM, tezAuto, + spec.tezAm().replicas(), patches, statuses, tezMetrics); } return new AutoscalingEvaluation(patches, statuses); } /** - * Scrape metrics for HS2 pods (used by LLAP/TezAM activation gate). + * Returns cached HS2 metrics (used by LLAP/TezAM activation gate). + * Non-blocking — reads from the background-scraper cache. */ - public List scrapeHs2Metrics(HiveCluster cluster) { + public List getHs2MetricsFromCache(HiveCluster cluster) { String namespace = cluster.getMetadata().getNamespace(); - Map selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); - int port = cluster.getSpec().hiveServer2().autoscaling().metricsPort(); - return scraper.scrape(namespace, selector, port); - } - - private void evaluateComponent(HiveCluster cluster, KubernetesClient client, - String namespace, String clusterName, String component, - AutoscalingSpec autoscaling, int maxReplicas, - Map patches, Map statuses) { - evaluateComponent(cluster, client, namespace, clusterName, component, - autoscaling, maxReplicas, patches, statuses, null); + String clusterName = cluster.getMetadata().getName(); + String key = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_HIVESERVER2; + int maxStale = cluster.getSpec().hiveServer2().autoscaling().metricsScrapeIntervalSeconds() * 3; + return metricsCache.getOrEmpty(key, maxStale); } private void evaluateComponent(HiveCluster cluster, KubernetesClient client, String namespace, String clusterName, String component, AutoscalingSpec autoscaling, int maxReplicas, Map patches, Map statuses, - List preScrapedMetrics) { + List metrics) { int currentReplicas = getCurrentReplicas(client, namespace, clusterName, component); String key = namespace + "/" + clusterName + "/" + component; - List metrics; - if (preScrapedMetrics != null) { - metrics = preScrapedMetrics; - } else { - Map selector = Labels.selectorForComponent(cluster, component); - metrics = scraper.scrape(namespace, selector, autoscaling.metricsPort()); - } - // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), // not their own pod metrics. Allow evaluation even with 0 own pods. boolean usesHs2Activation = ConfigUtils.COMPONENT_LLAP.equals(component) @@ -248,8 +329,11 @@ private int getCurrentReplicas(KubernetesClient client, String namespace, /** * Patches each pod's deletion cost annotation based on its active session count. * Kubernetes uses this during scale-down to kill idle pods first (lower cost = killed first). + *

+ * Only meaningful for Deployments (HS2, Metastore) — the ReplicaSet controller + * respects this annotation. StatefulSets ignore it and always terminate by ordinal. */ - private void updatePodDeletionCost(KubernetesClient client, String namespace, + private void updateDeploymentPodDeletionCost(KubernetesClient client, String namespace, List metrics, String metricName) { for (PodMetrics pm : metrics) { int sessions = pm.metrics().getOrDefault(metricName, 0.0).intValue(); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java index 87ed2a430593..e530f5adae66 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java @@ -57,7 +57,7 @@ public int computeDesiredReplicas(List podMetrics, // Activation gate: check if HS2 has any open sessions. // If scrape returns empty but LLAP has running pods, treat as "unknown" and preserve. // This prevents spurious scale-to-zero from transient scrape failures after operator restart. - List hs2Metrics = orchestrator.scrapeHs2Metrics(cluster); + List hs2Metrics = orchestrator.getHs2MetricsFromCache(cluster); Boolean sessionsDetected = detectHs2Sessions(hs2Metrics); if (sessionsDetected == null && !podMetrics.isEmpty()) { // HS2 scrape returned no data but LLAP is running — hold current state diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java new file mode 100644 index 000000000000..75b230ef2d64 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java @@ -0,0 +1,72 @@ +/* + * 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.hive.kubernetes.operator.autoscaling; + +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Thread-safe cache for scraped Prometheus metrics from pods. + * Entries become stale after a configurable duration (typically 3x the scrape interval) + * and are treated as absent when read. + */ +public class MetricsCache { + + private record CachedResult(List metrics, Instant scrapedAt) {} + + private final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + + /** + * Stores scraped metrics for a component. + * + * @param key format: "namespace/clusterName/component" + * @param metrics the scraped pod metrics + */ + public void put(String key, List metrics) { + cache.put(key, new CachedResult(metrics, Instant.now())); + } + + /** + * Returns cached metrics if present and not stale, otherwise an empty list. + * + * @param key format: "namespace/clusterName/component" + * @param maxStaleSecs maximum age in seconds before the entry is considered stale + * @return the cached metrics, or an empty list if absent or stale + */ + public List getOrEmpty(String key, int maxStaleSecs) { + CachedResult result = cache.get(key); + if (result == null) { + return Collections.emptyList(); + } + if (Instant.now().isAfter(result.scrapedAt().plusSeconds(maxStaleSecs))) { + return Collections.emptyList(); + } + return result.metrics(); + } + + /** + * Removes all entries whose key starts with the given prefix. + * Used for cleanup when a HiveCluster is deleted. + */ + public void removeByPrefix(String prefix) { + cache.keySet().removeIf(k -> k.startsWith(prefix)); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java index e09b6b9ca2ef..4e9fa302a715 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java @@ -52,7 +52,7 @@ public TezAmScalingStrategy(HiveClusterAutoscaler orchestrator, HiveCluster clus public int computeDesiredReplicas(List podMetrics, AutoscalingSpec autoscaling, int maxReplicas) { - List hs2Metrics = orchestrator.scrapeHs2Metrics(cluster); + List hs2Metrics = orchestrator.getHs2MetricsFromCache(cluster); // Activation gate: if HS2 scrape returns no data but TezAM has running pods, // treat as "unknown" and preserve current state to avoid spurious scale-to-zero. diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java index fa1f2abbdc9c..5a3e6cdc493d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -30,8 +30,8 @@ /** * Unified PodDisruptionBudget dependent resource for all Hive components. - * Ensures at least one pod remains available during voluntary disruptions - * (scale-down, node drain, rolling updates). + * Uses maxUnavailable=1 to allow at most one pod to be disrupted at a time + * while still permitting node drains when replicas=1. *

* Subclassed per component (HS2, Metastore, LLAP, TezAM) only to satisfy * JOSDK's requirement for distinct no-arg-constructible classes in the workflow. @@ -62,7 +62,7 @@ protected PodDisruptionBudget desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, component)) .endMetadata() .withNewSpec() - .withMinAvailable(new IntOrString(1)) + .withMaxUnavailable(new IntOrString(1)) .withNewSelector() .withMatchLabels(Labels.selectorForComponent(hiveCluster, component)) .endSelector() diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java index 31bead97718d..4d7a097c8a1d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -33,11 +33,15 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; import io.javaoperatorsdk.operator.api.reconciler.Context; +import io.javaoperatorsdk.operator.api.reconciler.Cleaner; import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration; +import io.javaoperatorsdk.operator.api.reconciler.DeleteControl; import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; import io.javaoperatorsdk.operator.api.reconciler.Reconciler; import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; +import org.apache.hive.kubernetes.operator.autoscaling.BackgroundMetricsScraper; import org.apache.hive.kubernetes.operator.autoscaling.HiveClusterAutoscaler; +import org.apache.hive.kubernetes.operator.autoscaling.MetricsCache; import org.apache.hive.kubernetes.operator.autoscaling.MetricsScraper; import org.apache.hive.kubernetes.operator.autoscaling.PodMetrics; import org.apache.hive.kubernetes.operator.model.HiveCluster; @@ -55,11 +59,13 @@ * Orchestrates all dependent resources with proper dependency ordering. */ @ControllerConfiguration -public class HiveClusterReconciler implements Reconciler { +public class HiveClusterReconciler + implements Reconciler, Cleaner { private static final Logger LOG = LoggerFactory.getLogger(HiveClusterReconciler.class); private volatile HiveClusterAutoscaler autoscaler; + private volatile BackgroundMetricsScraper bgScraper; @Override public UpdateControl reconcile(HiveCluster resource, Context context) { @@ -153,7 +159,9 @@ public UpdateControl reconcile(HiveCluster resource, Context reconcile(HiveCluster resource, Context context) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + if (autoscaler != null) { + autoscaler.cleanupCluster(ns, name); + } + if (bgScraper != null) { + bgScraper.unregisterCluster(ns, name); + } + LOG.info("Cleaned up autoscaler state for deleted cluster {}/{}", ns, name); + return DeleteControl.defaultDelete(); + } + @Override public ErrorStatusUpdateControl updateErrorStatus(HiveCluster resource, Context context, Exception e) { @@ -449,7 +471,10 @@ private void applyAutoscalingStatuses(HiveClusterStatus status, private HiveClusterAutoscaler getOrCreateAutoscaler(KubernetesClient client) { if (autoscaler == null) { - autoscaler = new HiveClusterAutoscaler(new MetricsScraper(client)); + MetricsScraper scraper = new MetricsScraper(client); + MetricsCache metricsCache = new MetricsCache(); + bgScraper = new BackgroundMetricsScraper(scraper, metricsCache); + autoscaler = new HiveClusterAutoscaler(scraper, bgScraper, metricsCache); } return autoscaler; } @@ -597,7 +622,7 @@ private boolean isClusterIdle(HiveCluster resource, KubernetesClient client) { // HS2 must have 0 open sessions. // If metrics scrape fails (empty list), assume NOT idle to prevent accidental suspend. HiveClusterAutoscaler scaler = getOrCreateAutoscaler(client); - List hs2Metrics = scaler.scrapeHs2Metrics(resource); + List hs2Metrics = scaler.getHs2MetricsFromCache(resource); if (hs2Metrics.isEmpty()) { LOG.debug("Idle check: HS2 metrics unavailable, assuming not idle"); return false; From 3c0db8e7f7e78f02acdc1fbe74dcec609f07990d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Fri, 12 Jun 2026 13:41:34 +0530 Subject: [PATCH 17/22] Cleanup --- .../autoscaling/TezAmScalingStrategy.java | 41 +++---------------- .../kubernetes/operator/util/ConfigUtils.java | 6 --- 2 files changed, 6 insertions(+), 41 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java index 4e9fa302a715..d2863102b098 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java @@ -22,16 +22,12 @@ import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; -import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Scaling strategy for Tez Application Master. - * TezAM scaling tracks HS2 session demand: - * - Trigger 1 (concurrent): sum(hs2_open_sessions) — each session may need a TezAM - * - Trigger 2 (pre-warm): count(hs2_pods_with_sessions) * sessions_per_queue - * desired = max(trigger1, trigger2) + * TezAM scaling tracks HS2 session demand: desired = ceil(sum(hs2_open_sessions)). *

* Activation gate: only scale if HS2 has open sessions. */ @@ -63,14 +59,9 @@ public int computeDesiredReplicas(List podMetrics, } double totalSessions = 0; - int podsWithSessions = 0; for (PodMetrics pm : hs2Metrics) { - double sessions = pm.metrics().getOrDefault( + totalSessions += pm.metrics().getOrDefault( HiveServer2ScalingStrategy.METRIC_OPEN_SESSIONS, 0.0); - totalSessions += sessions; - if (sessions > 0) { - podsWithSessions++; - } } if (totalSessions <= 0) { @@ -81,31 +72,11 @@ public int computeDesiredReplicas(List podMetrics, lastMetric = (int) totalSessions; - // Trigger 1: concurrent demand — total open sessions (1 TezAM per session) - int concurrentDemand = (int) Math.ceil(totalSessions); - - // Trigger 2: pre-warm — only if hive.server2.tez.initialize.default.sessions is true. - // When true, each HS2 pod pre-warms sessionsPerQueue TezAMs at startup. - // When false, no pre-warming happens — scale purely on concurrent session demand. - int prewarmDemand = 0; - boolean initSessions = ConfigUtils.getBoolean( - cluster.getSpec().hiveServer2().configOverrides(), - ConfigUtils.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_KEY, - ConfigUtils.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_DEFAULT); - if (initSessions) { - int sessionsPerQueue = ConfigUtils.getInt( - cluster.getSpec().hiveServer2().configOverrides(), - ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY, - null, ConfigUtils.HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT); - prewarmDemand = podsWithSessions * sessionsPerQueue; - } - - int desired = Math.max(concurrentDemand, prewarmDemand); + // Scale based on concurrent demand — one TezAM per open HS2 session + int desired = (int) Math.ceil(totalSessions); + desired = Math.min(desired, maxReplicas); - LOG.debug("[tezam] totalSessions={}, podsWithSessions={}, initDefaultSessions={}, " - + "concurrent={}, prewarm={}, desired={}", - totalSessions, podsWithSessions, initSessions, - concurrentDemand, prewarmDemand, desired); + LOG.debug("[tezam] totalSessions={}, desired={}", totalSessions, desired); return Math.max(desired, autoscaling.minReplicas()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java index 8761b23b298a..c3d824763c60 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -115,12 +115,6 @@ private ConfigUtils() { public static final String HIVE_METASTORE_URIS_KEY = "hive.metastore.uris"; - public static final String HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_KEY = "hive.server2.tez.sessions.per.default.queue"; - public static final int HIVE_SERVER2_TEZ_SESSIONS_PER_QUEUE_DEFAULT = 1; - - public static final String HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_KEY = - "hive.server2.tez.initialize.default.sessions"; - public static final boolean HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS_DEFAULT = true; public static final String HIVE_SERVER2_THRIFT_PORT_KEY = "hive.server2.thrift.port"; public static final int HIVE_SERVER2_THRIFT_PORT_DEFAULT = 10000; From be439fb297def42a935d90062deee3e23b974438 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 15 Jun 2026 16:05:38 +0530 Subject: [PATCH 18/22] Fix Log --- .../kubernetes/operator/autoscaling/LlapScalingStrategy.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java index e530f5adae66..074c2cac9fb1 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java @@ -73,9 +73,10 @@ public int computeDesiredReplicas(List podMetrics, // HS2 has sessions but LLAP has no pods yet — scale up to at least 1 if (podMetrics.isEmpty()) { - LOG.debug("[llap] HS2 has sessions but LLAP has 0 pods, scaling to 1"); + int minReplica = Math.max(1, autoscaling.minReplicas()); + LOG.debug("[llap] HS2 has sessions but LLAP has 0 pods, scaling to {}", minReplica); lastMetric = 0; - return Math.max(1, autoscaling.minReplicas()); + return minReplica; } // Compute average busy slots across all LLAP pods From 7af3e9806bab752e38125efb5d6666434a462d41 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 15 Jun 2026 19:45:44 +0530 Subject: [PATCH 19/22] CoPilot comments --- packaging/src/kubernetes/pom.xml | 6 +++++- .../autoscaling/BackgroundMetricsScraper.java | 4 +++- .../autoscaling/HiveClusterAutoscaler.java | 3 +++ .../HiveServer2ScalingStrategy.java | 3 ++- .../autoscaling/LlapScalingStrategy.java | 3 ++- .../autoscaling/MetastoreScalingStrategy.java | 3 ++- .../dependent/HiveDependentResource.java | 20 +++++++++++-------- .../HiveServer2DeploymentDependent.java | 4 ++-- .../dependent/LlapStatefulSetDependent.java | 4 ++-- 9 files changed, 33 insertions(+), 17 deletions(-) diff --git a/packaging/src/kubernetes/pom.xml b/packaging/src/kubernetes/pom.xml index 6cb6d6ce0c24..d5197b8c870d 100644 --- a/packaging/src/kubernetes/pom.xml +++ b/packaging/src/kubernetes/pom.xml @@ -26,6 +26,10 @@ Kubernetes operator for managing Apache Hive clusters ../../.. + + 2.0.16 @@ -68,7 +72,7 @@ org.slf4j slf4j-api - 2.0.16 + ${slf4j2.version} org.apache.logging.log4j diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java index 26c2fc83d668..9c58292fb565 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java @@ -50,10 +50,12 @@ public class BackgroundMetricsScraper { private final ConcurrentHashMap registeredIntervals = new ConcurrentHashMap<>(); + private static final int SCRAPER_THREAD_POOL_SIZE = 4; + public BackgroundMetricsScraper(MetricsScraper scraper, MetricsCache cache) { this.scraper = scraper; this.cache = cache; - this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + this.scheduler = Executors.newScheduledThreadPool(SCRAPER_THREAD_POOL_SIZE, r -> { Thread t = new Thread(r, "hive-metrics-scraper"); t.setDaemon(true); return t; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index b091f397c485..bf68e01be86c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -340,6 +340,9 @@ private void updateDeploymentPodDeletionCost(KubernetesClient client, String nam try { client.pods().inNamespace(namespace).withName(pm.podName()) .edit(pod -> { + if (pod.getMetadata().getAnnotations() == null) { + pod.getMetadata().setAnnotations(new java.util.HashMap<>()); + } pod.getMetadata().getAnnotations() .put("controller.kubernetes.io/pod-deletion-cost", String.valueOf(sessions)); return pod; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java index 72239d8e6041..92c826b29af4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveServer2ScalingStrategy.java @@ -52,7 +52,8 @@ public int computeDesiredReplicas(List podMetrics, return safeMinReplicas; } - int desired = (int) Math.ceil(totalSessions / autoscaling.scaleUpThreshold()); + int threshold = Math.max(1, autoscaling.scaleUpThreshold()); + int desired = (int) Math.ceil(totalSessions / threshold); return Math.max(desired, safeMinReplicas); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java index 074c2cac9fb1..598831b1aed2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java @@ -104,7 +104,8 @@ public int computeDesiredReplicas(List podMetrics, LOG.debug("[llap] avgBusy={}, threshold={}", String.format("%.2f", avgBusy), autoscaling.scaleUpThreshold()); - int desired = (int) Math.ceil(avgBusy / autoscaling.scaleUpThreshold()); + int threshold = Math.max(1, autoscaling.scaleUpThreshold()); + int desired = (int) Math.ceil(avgBusy / threshold); return Math.max(desired, autoscaling.minReplicas()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java index dbac58d928fa..fb73bcc0baf7 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java @@ -90,7 +90,8 @@ public int computeDesiredReplicas(List podMetrics, LOG.debug("[metastore] API request rate: {}/s, threshold: {}", String.format("%.2f", rate), autoscaling.scaleUpThreshold()); - int desired = (int) Math.ceil(rate / autoscaling.scaleUpThreshold()); + int threshold = Math.max(1, autoscaling.scaleUpThreshold()); + int desired = (int) Math.ceil(rate / threshold); return Math.max(desired, autoscaling.minReplicas()); } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 9f7a345daffd..8ebba672b9f4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -226,20 +226,20 @@ protected String getComponentName() { protected static String buildDrainScript( String startupMessage, String metricName, String varName, String idleMessage, int sleepSeconds, int maxRetries, - List prefixCommands) { + List prefixCommands, int metricsPort) { List lines = new ArrayList<>(); lines.add("#!/bin/bash"); if (prefixCommands != null) { lines.addAll(prefixCommands); } lines.add("echo '[preStop] " + startupMessage - + " (polling localhost:9404/metrics)...'"); + + " (polling localhost:" + metricsPort + "/metrics)...'"); lines.add("RETRIES=0"); lines.add("while true; do"); - lines.add(" RESPONSE=$(curl -sf http://localhost:9404/metrics)"); + lines.add(" RESPONSE=$(curl -sf http://localhost:" + metricsPort + "/metrics)"); lines.add(" if [ $? -ne 0 ]; then"); lines.add(" RETRIES=$((RETRIES+1))"); - lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\""); + lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port " + metricsPort + " (attempt $RETRIES)\""); lines.add(" if [ $RETRIES -ge " + maxRetries + " ]; then"); lines.add(" echo '[preStop] JMX Exporter not responding after " + (maxRetries * sleepSeconds) + "s. Proceeding with shutdown.'"); @@ -292,17 +292,18 @@ protected static String buildDualMetricDrainScript( String metricGrepA, String varNameA, String metricGrepB, String varNameB, String notFoundWarning, String idleMessage, - String waitingFormat, int sleepSeconds, int maxRetries) { + String waitingFormat, int sleepSeconds, int maxRetries, + int metricsPort) { List lines = new ArrayList<>(); lines.add("#!/bin/bash"); lines.add("echo '[preStop] " + startupMessage - + " (polling localhost:9404/metrics)...'"); + + " (polling localhost:" + metricsPort + "/metrics)...'"); lines.add("RETRIES=0"); lines.add("while true; do"); - lines.add(" RESPONSE=$(curl -sf http://localhost:9404/metrics)"); + lines.add(" RESPONSE=$(curl -sf http://localhost:" + metricsPort + "/metrics)"); lines.add(" if [ $? -ne 0 ]; then"); lines.add(" RETRIES=$((RETRIES+1))"); - lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port 9404 (attempt $RETRIES)\""); + lines.add(" echo \"[preStop] ERROR: JMX Exporter unreachable on port " + metricsPort + " (attempt $RETRIES)\""); lines.add(" if [ $RETRIES -ge " + maxRetries + " ]; then"); lines.add(" echo '[preStop] JMX Exporter not responding after " + (maxRetries * sleepSeconds) + "s. Proceeding with shutdown.'"); @@ -667,6 +668,9 @@ protected static void applyAutoscalingLifecycle( private static void applyPrometheusScrapeAnnotations( io.fabric8.kubernetes.api.model.ObjectMeta podMetadata, int scrapeIntervalSeconds) { + if (podMetadata.getAnnotations() == null) { + podMetadata.setAnnotations(new java.util.HashMap<>()); + } podMetadata.getAnnotations().put("prometheus.io/scrape", "true"); podMetadata.getAnnotations().put("prometheus.io/port", String.valueOf(ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT)); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java index bf18ed7a7236..a7809e1ef0ff 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -265,7 +265,7 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); - // Graceful scale-down: deregister from ZK, then poll JMX Exporter (port 9404) for sessions. + // Graceful scale-down: deregister from ZK, then poll JMX Exporter for sessions. if (autoscaling.isEnabled()) { List zkDeregister = List.of( "echo '[preStop] Deregistering HiveServer2 from ZooKeeper...'", @@ -275,7 +275,7 @@ protected Deployment desired(HiveCluster hiveCluster, "Waiting for open sessions to drain", "hs2_open_sessions", "SESSIONS", "All sessions drained. Shutting down.", - 5, 6, zkDeregister); + 5, 6, zkDeregister, autoscaling.metricsPort()); applyAutoscalingLifecycle( deployment.getSpec().getTemplate().getSpec(), deployment.getSpec().getTemplate().getMetadata(), diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 380a7bea7967..3cd937329c97 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -199,7 +199,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); - // Graceful scale-down: poll JMX Exporter (port 9404) until all executors idle. + // Graceful scale-down: poll JMX Exporter until all executors idle. if (autoscaling.isEnabled()) { String preStopScript = buildDualMetricDrainScript( "Waiting for LLAP executors to become idle", @@ -208,7 +208,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, "LLAP executor metrics not found. JMX Exporter may not be configured.", "All executors idle. Shutting down.", "Executors available=$AVAILABLE / total=$TOTAL \u2014 waiting...", - 10, 6); + 10, 6, autoscaling.metricsPort()); applyAutoscalingLifecycle( statefulSet.getSpec().getTemplate().getSpec(), statefulSet.getSpec().getTemplate().getMetadata(), From b2fe4770b8df034dcbb0f6ef2ae2c1e0143121d3 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 15 Jun 2026 20:11:11 +0530 Subject: [PATCH 20/22] Add Parallel --- .../kubernetes/operator/dependent/LlapStatefulSetDependent.java | 1 + .../kubernetes/operator/dependent/TezAmStatefulSetDependent.java | 1 + 2 files changed, 2 insertions(+) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java index 3cd937329c97..9862d37d80e2 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -168,6 +168,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .endMetadata() .withNewSpec() .withReplicas(replicas) + .withPodManagementPolicy("Parallel") .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java index fd81af716e6a..56aa77c05db4 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -149,6 +149,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .endMetadata() .withNewSpec() .withReplicas(replicas) + .withPodManagementPolicy("Parallel") .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) From 939ba2caca7686cbb015a8d9f8dab90788db1a20 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 17 Jun 2026 22:24:13 +0530 Subject: [PATCH 21/22] Fix Some Sonar issues --- .../operator/autoscaling/HiveClusterAutoscaler.java | 2 -- .../operator/autoscaling/LlapScalingStrategy.java | 6 ++++-- .../operator/autoscaling/MetastoreScalingStrategy.java | 8 +++++--- .../operator/dependent/HiveDependentResource.java | 9 +++++---- .../hive/kubernetes/operator/model/HiveClusterSpec.java | 2 +- .../kubernetes/operator/model/spec/AutoSuspendSpec.java | 4 ++-- 6 files changed, 17 insertions(+), 14 deletions(-) diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index bf68e01be86c..37fda2881fbd 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -75,7 +75,6 @@ public static void setManagedReplicas(String namespace, String clusterName, private record PendingScaleDown(int targetReplicas, Instant annotatedAt) {} - private final MetricsScraper scraper; private final BackgroundMetricsScraper bgScraper; private final MetricsCache metricsCache; // Key: "namespace/clusterName/component" @@ -90,7 +89,6 @@ private record PendingScaleDown(int targetReplicas, Instant annotatedAt) {} public HiveClusterAutoscaler(MetricsScraper scraper, BackgroundMetricsScraper bgScraper, MetricsCache metricsCache) { - this.scraper = scraper; this.bgScraper = bgScraper; this.metricsCache = metricsCache; } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java index 598831b1aed2..1be30a94b1f6 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/LlapScalingStrategy.java @@ -101,8 +101,10 @@ public int computeDesiredReplicas(List podMetrics, return Math.max(1, autoscaling.minReplicas()); } - LOG.debug("[llap] avgBusy={}, threshold={}", String.format("%.2f", avgBusy), - autoscaling.scaleUpThreshold()); + if (LOG.isDebugEnabled()) { + LOG.debug("[llap] avgBusy={}, threshold={}", String.format("%.2f", avgBusy), + autoscaling.scaleUpThreshold()); + } int threshold = Math.max(1, autoscaling.scaleUpThreshold()); int desired = (int) Math.ceil(avgBusy / threshold); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java index fb73bcc0baf7..fc4b9e30f499 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetastoreScalingStrategy.java @@ -78,7 +78,7 @@ public int computeDesiredReplicas(List podMetrics, // Store current state for next evaluation previousCounters.clear(); - previousCounters.put("_total", currentTotal); + previousCounters.put(API_COUNTER_SUFFIX, currentTotal); previousTimestampMs = now; lastMetric = (int) Math.round(rate); @@ -87,8 +87,10 @@ public int computeDesiredReplicas(List podMetrics, return autoscaling.minReplicas(); } - LOG.debug("[metastore] API request rate: {}/s, threshold: {}", - String.format("%.2f", rate), autoscaling.scaleUpThreshold()); + if (LOG.isDebugEnabled()) { + LOG.debug("[metastore] API request rate: {}/s, threshold: {}", + String.format("%.2f", rate), autoscaling.scaleUpThreshold()); + } int threshold = Math.max(1, autoscaling.scaleUpThreshold()); int desired = (int) Math.ceil(rate / threshold); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java index 8ebba672b9f4..2315b455d760 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -840,16 +840,17 @@ private static String buildJmxExporterConfig(String component) { // causing CPU spikes and GC pressure on the LLAP JVM. // Internal format: Hadoop><>Attribute // Separate rules per attribute — JMX Exporter 1.x caches per-bean, not per-attribute. - sb.append("- pattern: 'Hadoop<>ExecutorNumQueuedRequests'\n"); + String llapBean = "Hadoop<>"; + sb.append("- pattern: '").append(llapBean).append("ExecutorNumQueuedRequests'\n"); sb.append(" name: hadoop_llapdaemon_executornumqueuedrequests\n"); sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsConfigured'\n"); + sb.append("- pattern: '").append(llapBean).append("ExecutorNumExecutorsConfigured'\n"); sb.append(" name: hadoop_llapdaemon_executornumexecutorsconfigured\n"); sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsAvailable'\n"); + sb.append("- pattern: '").append(llapBean).append("ExecutorNumExecutorsAvailable'\n"); sb.append(" name: hadoop_llapdaemon_executornumexecutorsavailable\n"); sb.append(" type: GAUGE\n"); - sb.append("- pattern: 'Hadoop<>ExecutorNumExecutors'\n"); + sb.append("- pattern: '").append(llapBean).append("ExecutorNumExecutors'\n"); sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); sb.append(" type: GAUGE\n"); break; diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java index a6d9c709880c..c527a7debbd0 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java @@ -98,7 +98,7 @@ public record HiveClusterSpec( volumes = volumes != null ? volumes : List.of(); volumeMounts = volumeMounts != null ? volumeMounts : List.of(); autoSuspend = autoSuspend != null ? autoSuspend : new AutoSuspendSpec(false, 15, true); - suspend = suspend != null ? suspend : false; + suspend = suspend != null && suspend; } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java index 196d57e8bc86..a08f9c88231a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java @@ -43,9 +43,9 @@ public record AutoSuspendSpec( Boolean includeMetastore) { public AutoSuspendSpec { - enabled = enabled != null ? enabled : false; + enabled = enabled != null && enabled; idleTimeoutMinutes = idleTimeoutMinutes != null ? idleTimeoutMinutes : 15; - includeMetastore = includeMetastore != null ? includeMetastore : true; + includeMetastore = includeMetastore == null || includeMetastore; } public boolean isEnabled() { From 49c8ec4cb3528a2e9ae70210d6a0794c8d44f889 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Thu, 18 Jun 2026 08:41:02 +0530 Subject: [PATCH 22/22] Review Comments --- packaging/src/kubernetes/pom.xml | 4 ++++ .../autoscaling/BackgroundMetricsScraper.java | 2 +- .../operator/autoscaling/HiveClusterAutoscaler.java | 11 ++++++++--- .../kubernetes/operator/autoscaling/MetricsCache.java | 6 ++---- .../operator/autoscaling/PrometheusTextParser.java | 4 +++- 5 files changed, 18 insertions(+), 9 deletions(-) diff --git a/packaging/src/kubernetes/pom.xml b/packaging/src/kubernetes/pom.xml index d5197b8c870d..f9e7bd046de8 100644 --- a/packaging/src/kubernetes/pom.xml +++ b/packaging/src/kubernetes/pom.xml @@ -52,6 +52,10 @@ kubernetes-httpclient-vertx ${fabric8.version} + + org.apache.commons + commons-lang3 + io.github.java-diff-utils java-diff-utils diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java index 9c58292fb565..0e13136b07c5 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java @@ -76,7 +76,7 @@ public BackgroundMetricsScraper(MetricsScraper scraper, MetricsCache cache) { public void registerOrUpdate(String namespace, String clusterName, String component, Map selector, int metricsPort, int intervalSecs) { - String key = namespace + "/" + clusterName + "/" + component; + String key = HiveClusterAutoscaler.cacheKey(namespace, clusterName, component); Integer existing = registeredIntervals.get(key); if (existing != null && existing == intervalSecs) { return; // Already registered with same interval diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java index 37fda2881fbd..2f0e661a27ca 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -56,12 +56,17 @@ public record AutoscalingEvaluation( private static final ConcurrentHashMap MANAGED_REPLICAS = new ConcurrentHashMap<>(); + /** Builds the cache key used for per-component state maps. */ + static String cacheKey(String namespace, String clusterName, String component) { + return namespace + "/" + clusterName + "/" + component; + } + /** * Returns the autoscaler-managed replica count for a component, or null if the * autoscaler hasn't made a decision yet (e.g., first reconcile before evaluation runs). */ public static Integer getManagedReplicas(String namespace, String clusterName, String component) { - return MANAGED_REPLICAS.get(namespace + "/" + clusterName + "/" + component); + return MANAGED_REPLICAS.get(cacheKey(namespace, clusterName, component)); } /** @@ -70,7 +75,7 @@ public static Integer getManagedReplicas(String namespace, String clusterName, S */ public static void setManagedReplicas(String namespace, String clusterName, String component, int replicas) { - MANAGED_REPLICAS.put(namespace + "/" + clusterName + "/" + component, replicas); + MANAGED_REPLICAS.put(cacheKey(namespace, clusterName, component), replicas); } private record PendingScaleDown(int targetReplicas, Instant annotatedAt) {} @@ -249,7 +254,7 @@ private void evaluateComponent(HiveCluster cluster, KubernetesClient client, int currentReplicas = getCurrentReplicas(client, namespace, clusterName, component); - String key = namespace + "/" + clusterName + "/" + component; + String key = cacheKey(namespace, clusterName, component); // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), // not their own pod metrics. Allow evaluation even with 0 own pods. diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java index 75b230ef2d64..687ba739de86 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java @@ -53,10 +53,8 @@ public void put(String key, List metrics) { */ public List getOrEmpty(String key, int maxStaleSecs) { CachedResult result = cache.get(key); - if (result == null) { - return Collections.emptyList(); - } - if (Instant.now().isAfter(result.scrapedAt().plusSeconds(maxStaleSecs))) { + if (result == null + || Instant.now().isAfter(result.scrapedAt().plusSeconds(maxStaleSecs))) { return Collections.emptyList(); } return result.metrics(); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java index 8940a80b1934..babee17aa77d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -24,6 +24,8 @@ import java.util.HashMap; import java.util.Map; +import org.apache.commons.lang3.StringUtils; + /** * Parses Prometheus text exposition format (from JMX Exporter /metrics). * Only extracts metric name → value pairs; labels are stripped. @@ -53,7 +55,7 @@ public static Map parseWithLabels(String body) { private static Map doParse(String body, boolean keepLabels) { Map result = new HashMap<>(); - if (body == null || body.isEmpty()) { + if (StringUtils.isEmpty(body)) { return result; } try (BufferedReader reader = new BufferedReader(new StringReader(body))) {