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 1fc11623240c..54e58393fb85 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -505,19 +505,350 @@ kubectl get hiveclusters kubectl describe hivecluster hive ``` +--- + +## Autoscaling + +The operator supports metric-based autoscaling for all four Hive components using +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 + +- No external dependencies — the operator handles all scaling decisions internally + +### How It Works + +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. 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 +> 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 three independent timing controls: + +| Timer | Config Field | Default | Purpose | +|-------|-------------|---------|---------| +| **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 → 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 ~10-20s (one scrape cycle). For HS2 with `scaleUpStabilizationSeconds: 60`, expect ~70s. + +### Per-Component Scaling Logic + +| 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 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 with sessions) × hive.server2.tez.sessions.per.default.queue` (default 1): + ensures every active HS2 pod has enough TezAM sessions pre-claimed from ZooKeeper. + +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 (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. HS2 reports hs2_open_sessions = 0 for scaleDownStabilization + → operator scales HS2 to minReplicas (>= 1) + + 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 (LLAP/TezAM) + + 1. Beeline connects to HS2 (always running, at least 1 pod) + + 2. HS2 reports hs2_open_sessions > 0 via JMX Exporter + + 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 + +``` + +**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. + +**Component-specific behavior:** + +| Component | minReplicas | Scale-to-Zero Trigger | Wake Trigger | +|-----------|-------------|----------------------|--------------| +| **HS2** | 1 | N/A (always running) | N/A | +| **HMS** | 1 | Never (always running) | N/A | +| **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 + +**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 \ + --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.metastore.autoscaling.enabled=true \ + --set cluster.llap.autoscaling.enabled=true \ + --set cluster.tezAm.autoscaling.enabled=true +``` + +**Values file (for customizing beyond defaults):** + +```yaml +# values-autoscaling.yaml — only override what you need +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 maxReplicas when autoscaling is enabled + autoscaling: + enabled: true + # minReplicas: 1 # default — always keep at least 1 HS2 running + # scaleUpThreshold: 80 # default — avg open sessions per pod triggering scale-up + # 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: 1 # default — always keep at least 1 metastore running + # scaleUpThreshold: 75 # default — API request rate (req/s) triggering scale-up + # 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 — operator scrape interval + + llap: + replicas: 8 # Acts as maxReplicas when autoscaling is enabled + autoscaling: + enabled: true + # minReplicas: 0 # default — scale to zero when no HS2 sessions + # scaleUpThreshold: 1 # default — total busy slots (queued+running) triggering scale-up + # 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 — operator 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: 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 — operator scrape interval (lower = faster reaction) +``` + +```bash +helm install hive ./helm/hive-operator -f values-autoscaling.yaml +``` + +When autoscaling is enabled, the operator automatically: +- Deploys the JMX Exporter javaagent (port 9404, `/metrics`) +- Enables `hive.server2.metrics.enabled` / `metastore.metrics.enabled` (JMX reporter) +- 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 +- LLAP/TezAM use HS2 metrics as activation gate (only scale when HS2 has sessions) + +**JMX Metrics Scraped by Operator (per component):** + +| 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) | +| **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 | + +### Enabling Autoscaling — Example + +To enable autoscaling for HS2 and Metastore: + +```yaml +cluster: + hiveServer2: + replicas: 4 # max replicas ceiling + autoscaling: + enabled: true + scaleUpThreshold: 1 # scale up when total sessions > 1 + minReplicas: 1 # always keep at least 1 HS2 pod running + + metastore: + replicas: 3 # max replicas ceiling + autoscaling: + enabled: true + minReplicas: 1 # always keep at least 1 running + scaleUpThreshold: 75 # API requests/sec threshold +``` + +> **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). + +### 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 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.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. | + +--- + ## 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" ``` +### LLAP/TezAM Scale-to-Zero Behavior + +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. + +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 +> `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. + +> **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 @@ -620,6 +951,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 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.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` | How often the operator polls JMX metrics from pods. Lower = faster reaction time. | + --- ## Upgrade and Uninstall @@ -659,11 +1002,13 @@ helm install hive ./helm/hive-operator -f my-values.yaml ### Remove Everything (including dependencies) ```bash -helm uninstall hive -kubectl delete crd hiveclusters.hive.apache.org -helm uninstall ozone postgres zookeeper --ignore-not-found -kubectl delete pvc data-zookeeper-0 --ignore-not-found -kubectl delete pvc data-postgres-postgresql-0 --ignore-not-found +kubectl delete hivecluster --all -A --wait=false --ignore-not-found +helm uninstall hive --ignore-not-found +kubectl delete crd hiveclusters.hive.apache.org --wait=false --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 ``` 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..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 @@ -44,6 +44,53 @@ spec: hiveServer2: description: HiveServer2 component configuration properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 3600 + description: 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. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + 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 (LLAP, TezAM only; HS2 minimum\ + \ is 1)" + type: integer + scaleDownStabilizationSeconds: + default: 600 + description: Stabilization window in seconds for scale-down + 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. Picks the highest recommendation within this + window to prevent flapping. + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ 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 + type: object configOverrides: additionalProperties: type: string @@ -152,6 +199,53 @@ spec: llap: description: LLAP daemon configuration. Enabled by default. properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 3600 + description: 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. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + 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 (LLAP, TezAM only; HS2 minimum\ + \ is 1)" + type: integer + scaleDownStabilizationSeconds: + default: 600 + description: Stabilization window in seconds for scale-down + 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. Picks the highest recommendation within this + window to prevent flapping. + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ 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 + type: object configOverrides: additionalProperties: type: string @@ -235,6 +329,53 @@ spec: metastore: description: Metastore component configuration properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 3600 + description: 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. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + 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 (LLAP, TezAM only; HS2 minimum\ + \ is 1)" + type: integer + scaleDownStabilizationSeconds: + default: 600 + description: Stabilization window in seconds for scale-down + 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. Picks the highest recommendation within this + window to prevent flapping. + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ 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 + type: object configOverrides: additionalProperties: type: string @@ -371,6 +512,53 @@ spec: tezAm: description: Tez Application Master configuration. Enabled by default. properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + enabled: + default: false + description: Whether autoscaling is enabled for this component + type: boolean + gracePeriodSeconds: + default: 3600 + description: 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. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + 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 (LLAP, TezAM only; HS2 minimum\ + \ is 1)" + type: integer + scaleDownStabilizationSeconds: + default: 600 + description: Stabilization window in seconds for scale-down + 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. Picks the highest recommendation within this + window to prevent flapping. + type: integer + scaleUpThreshold: + default: 80 + description: "Threshold that triggers scale-up (component-specific:\ + \ 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 + type: object configOverrides: additionalProperties: type: string @@ -476,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 @@ -485,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 @@ -494,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 @@ -505,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 d27e1fea8c6f..3b0eb0e8e40f 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"] @@ -46,7 +50,11 @@ 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"] + 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..f6178ab73a72 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -67,6 +67,16 @@ 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 }} + 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 }} externalUri: {{ .Values.cluster.metastore.externalUri | quote }} @@ -96,6 +106,16 @@ 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 }} + 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: enabled: {{ .Values.cluster.llap.enabled }} @@ -120,6 +140,16 @@ 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 }} + 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 }} tezAm: @@ -146,6 +176,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 }} + 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 }} zookeeper: diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index b7d75930c5b2..2cc145f7839c 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -112,6 +112,17 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # 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 + minReplicas: 1 + 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" @@ -127,6 +138,16 @@ cluster: externalJars: [] extraVolumes: [] extraVolumeMounts: [] + # Autoscaling (operator-driven, no external dependencies) + # When enabled, 'replicas' above acts as the max replica ceiling + autoscaling: + enabled: false + minReplicas: 1 + scaleUpThreshold: 1 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 600 + gracePeriodSeconds: 300 + metricsScrapeIntervalSeconds: 10 # --------------------------------------------------------------------------- # LLAP — enabled by default for full-HA @@ -141,6 +162,17 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # 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 + minReplicas: 0 + scaleUpThreshold: 1 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 900 + gracePeriodSeconds: 600 + metricsScrapeIntervalSeconds: 10 # --------------------------------------------------------------------------- # TEZ AM — enabled by default for full-HA @@ -154,3 +186,15 @@ cluster: configOverrides: {} extraVolumes: [] extraVolumeMounts: [] + # 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 + # TezAM scales demand-based: max(totalSessions, hs2Pods * sessionsPerQueue) + # No scaleUpThreshold needed — scaling is 1:1 with session demand + autoscaling: + enabled: false + minReplicas: 0 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 600 + gracePeriodSeconds: 120 + metricsScrapeIntervalSeconds: 10 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/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..7a70a48c9e72 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -0,0 +1,245 @@ +/* + * 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()) { + 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, hs2Metrics); + } + + // 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) { + 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; + + 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. + 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; + } + } + + /** + * 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/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..3f8ab6527703 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -0,0 +1,137 @@ +/* + * 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.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; +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; + } + 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; + } + + /** + * 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; + } + 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, 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 + } + } + } catch (IOException e) { + // StringReader does not throw IOException + } + 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/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 cc2eb0de6de0..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 @@ -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; @@ -44,12 +45,15 @@ 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; 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; @@ -78,32 +82,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(); + }); } /** @@ -125,6 +138,208 @@ 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. + *

+ * 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). + */ + protected Integer resolveReplicaCount(P primary, Context

context, + AutoscalingSpec autoscaling, int staticReplicas, int initialReplicas) { + if (autoscaling == null || !autoscaling.isEnabled()) { + return staticReplicas; + } + 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; + } + + /** + * 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"); + // 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("pkill -f 'java.*org.apache' || true"); + lines.add("exit 0"); + 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"); + // 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("pkill -f 'java.*org.apache' || true"); + lines.add("exit 0"); + 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. @@ -235,8 +450,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. */ @@ -422,4 +637,225 @@ 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, + int metricsScrapeIntervalSeconds) { + 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); + 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"); + } + + /** + * 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"; + 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) + .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); + 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<>Count'\n"); + sb.append(" name: hive_metastore_open_connections\n"); + sb.append(" type: GAUGE\n"); + break; + case "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 "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/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java new file mode 100644 index 000000000000..e19413341c9b --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -0,0 +1,111 @@ +/* + * 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/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 ccb3048dea98..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 @@ -34,6 +34,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; @@ -55,6 +56,17 @@ public HiveServer2DeploymentDependent() { super(Deployment.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -125,21 +137,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).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("http") + .withContainerPort(hs2HttpPort).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("webui") + .withContainerPort(hs2WebUiPort).withProtocol("TCP").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(); @@ -155,8 +174,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( @@ -185,6 +204,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 +220,12 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + AutoscalingSpec hs2Autoscaling = hs2.autoscaling(); + int initialReplicas = hs2Autoscaling != null && hs2Autoscaling.isEnabled() + ? Math.max(1, hs2Autoscaling.minReplicas()) : hs2.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, hs2Autoscaling, hs2.replicas(), initialReplicas); + Deployment deployment = new DeploymentBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -201,7 +233,7 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(hs2.replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() @@ -233,21 +265,28 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); - 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()); + // Graceful scale-down: deregister from ZK, then poll JMX Exporter (port 9404) for sessions. + if (autoscaling.isEnabled()) { + List zkDeregister = List.of( + "echo '[preStop] Deregistering HiveServer2 from ZooKeeper...'", + "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", + "All sessions drained. Shutting down.", + 5, 6, zkDeregister); + applyAutoscalingLifecycle( + deployment.getSpec().getTemplate().getSpec(), + deployment.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } + 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/HiveServer2ServiceDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java deleted file mode 100644 index a9707ac0dfa6..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2ServiceDependent.java +++ /dev/null @@ -1,79 +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 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("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..edd048e8a322 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -0,0 +1,173 @@ +/* + * 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; + +/** + * 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()) + .withSessionAffinity("ClientIP") + .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/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 c8c044d22ce9..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 @@ -34,6 +34,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.LlapSpec; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; @@ -56,6 +57,17 @@ public LlapStatefulSetDependent() { super(StatefulSet.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -81,16 +93,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).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("shuffle").withContainerPort(15551).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("web").withContainerPort(15002).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("output").withContainerPort(15003).withProtocol("TCP").build()); Probe readinessProbe = buildTcpProbe(15004, llap.readinessProbe(), 15, 10, 3); @@ -106,8 +117,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(), @@ -115,11 +126,24 @@ 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))); + AutoscalingSpec llapAutoscaling = llap.autoscaling(); + int initialReplicas = llapAutoscaling != null && llapAutoscaling.isEnabled() + ? llapAutoscaling.minReplicas() : llap.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, llapAutoscaling, llap.replicas(), initialReplicas); + StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -127,7 +151,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(llap.replicas()) + .withReplicas(replicas) .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) @@ -159,20 +183,27 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); - 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()); + // Graceful scale-down: poll JMX Exporter (port 9404) until all executors idle. + if (autoscaling.isEnabled()) { + 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(), + autoscaling.metricsScrapeIntervalSeconds()); } + + 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 46a95426c969..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 @@ -36,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.DatabaseConfig; import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; @@ -56,6 +57,17 @@ public MetastoreDeploymentDependent() { super(Deployment.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected Deployment desired(HiveCluster hiveCluster, Context context) { @@ -77,12 +89,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).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .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); @@ -107,6 +118,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 +133,12 @@ protected Deployment desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getMetastoreSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + AutoscalingSpec msAutoscaling = spec.metastore().autoscaling(); + int initialReplicas = msAutoscaling != null && msAutoscaling.isEnabled() + ? Math.max(1, msAutoscaling.minReplicas()) : spec.metastore().replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, msAutoscaling, spec.metastore().replicas(), initialReplicas); + Deployment deployment = new DeploymentBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -122,7 +146,7 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(spec.metastore().replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() @@ -155,20 +179,26 @@ protected Deployment desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( deployment.getSpec().getTemplate().getSpec(), selectorLabels); - 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()); + // 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 = String.join("\n", + "#!/bin/bash", + "echo '[preStop] Sending SIGTERM to Metastore Java process...'", + "pkill -f 'java.*org.apache' || true", + "exit 0"); + applyAutoscalingLifecycle( + deployment.getSpec().getTemplate().getSpec(), + deployment.getSpec().getTemplate().getMetadata(), + preStopScript, autoscaling.gracePeriodSeconds(), + autoscaling.metricsScrapeIntervalSeconds()); } + + 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/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/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 5cc7a3f800f3..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 @@ -23,6 +23,7 @@ import java.util.Map; import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerPort; import io.fabric8.kubernetes.api.model.EnvVar; import io.fabric8.kubernetes.api.model.apps.StatefulSet; import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder; @@ -31,6 +32,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; @@ -57,6 +59,17 @@ public TezAmStatefulSetDependent() { super(StatefulSet.class); } + @Override + protected String getSecondaryResourceName(HiveCluster primary, + Context context) { + return resourceName(primary); + } + + @Override + protected String getComponentName() { + return COMPONENT; + } + @Override protected StatefulSet desired(HiveCluster hiveCluster, Context context) { @@ -98,8 +111,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() @@ -107,6 +120,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .endPersistentVolumeClaim() .build()); + List ports = new ArrayList<>(); List initContainers = new ArrayList<>(); addExternalJars(spec.image(), spec.externalJars(), initContainers, volumeMounts, volumes, envVars); @@ -120,6 +134,12 @@ protected StatefulSet desired(HiveCluster hiveCluster, HadoopXmlBuilder.buildXml(HiveConfigBuilder.getTezSite(spec)), HadoopXmlBuilder.buildXml(HiveConfigBuilder.getHadoopCoreSite(spec))); + AutoscalingSpec tezAmAutoscaling = tezAm.autoscaling(); + int initialReplicas = tezAmAutoscaling != null && tezAmAutoscaling.isEnabled() + ? tezAmAutoscaling.minReplicas() : tezAm.replicas(); + Integer replicas = resolveReplicaCount( + hiveCluster, context, tezAmAutoscaling, tezAm.replicas(), initialReplicas); + StatefulSet statefulSet = new StatefulSetBuilder() .withNewMetadata() .withName(resourceName(hiveCluster)) @@ -127,7 +147,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 +165,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) + .withPorts(ports) .withResources(buildResources(tezAm.resources())) .withVolumeMounts(volumeMounts) .endContainer() @@ -157,20 +178,10 @@ protected StatefulSet desired(HiveCluster hiveCluster, applySpreadAffinityIfAbsent( statefulSet.getSpec().getTemplate().getSpec(), selectorLabels); - 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/HiveServer2Precondition.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java deleted file mode 100644 index a36002dbf886..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/HiveServer2Precondition.java +++ /dev/null @@ -1,53 +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; - } - - int 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/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/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 7b3169f32043..000000000000 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/condition/MetastoreReadyCondition.java +++ /dev/null @@ -1,49 +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; - } - int 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/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/AutoscalingSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java new file mode 100644 index 000000000000..2ec6d5fda0c3 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -0,0 +1,71 @@ +/* + * 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. 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 (LLAP, TezAM only; HS2 minimum is 1)") + @Default("0") + Integer minReplicas, + @JsonPropertyDescription("Threshold that triggers scale-up (component-specific: " + + "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("Stabilization window in seconds for scale-up decisions. " + + "Picks the highest recommendation within this window to prevent flapping.") + @Default("60") + Integer scaleUpStabilizationSeconds, + @JsonPropertyDescription("Stabilization window in seconds for scale-down decisions. " + + "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. " + + "The pod terminates immediately once sessions/connections drain to 0; " + + "this value is only the upper safety cap.") + @Default("3600") + Integer gracePeriodSeconds, + @JsonPropertyDescription("How often (seconds) the operator scrapes JMX metrics from pods. " + + "Lower values make autoscaling react faster.") + @Default("10") + Integer metricsScrapeIntervalSeconds) { + + public AutoscalingSpec { + enabled = enabled != null ? enabled : false; + minReplicas = minReplicas != null ? minReplicas : 0; + scaleUpThreshold = scaleUpThreshold != null ? scaleUpThreshold : 80; + scaleUpStabilizationSeconds = scaleUpStabilizationSeconds != null ? scaleUpStabilizationSeconds : 60; + scaleDownStabilizationSeconds = scaleDownStabilizationSeconds != null ? scaleDownStabilizationSeconds : 600; + gracePeriodSeconds = gracePeriodSeconds != null ? gracePeriodSeconds : 3600; + metricsScrapeIntervalSeconds = metricsScrapeIntervalSeconds != null ? metricsScrapeIntervalSeconds : 10; + } + + 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..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 @@ -51,7 +51,9 @@ public record HiveServer2Spec( @JsonPropertyDescription("Readiness probe configuration") ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") - ProbeSpec livenessProbe) { + ProbeSpec livenessProbe, + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") + 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, 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 17ff5967ff9a..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 @@ -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 (operator-driven, no external dependencies)") + 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, 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 307c17221ee7..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 @@ -56,7 +56,9 @@ public record MetastoreSpec( @JsonPropertyDescription("Readiness probe configuration") ProbeSpec readinessProbe, @JsonPropertyDescription("Liveness probe configuration") - ProbeSpec livenessProbe) { + ProbeSpec livenessProbe, + @JsonPropertyDescription("Autoscaling configuration (operator-driven, no external dependencies)") + 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, 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 a0494c2c5e73..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 @@ -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 (operator-driven, no external dependencies)") + 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, 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 20332cb4127c..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 @@ -18,47 +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 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.HiveServer2ServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.LlapConfigMapDependent; -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.MetastoreServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.SchemaInitJobDependent; -import org.apache.hive.kubernetes.operator.dependent.ScratchPvcDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmServiceDependent; -import org.apache.hive.kubernetes.operator.dependent.TezAmStatefulSetDependent; -import org.apache.hive.kubernetes.operator.dependent.condition.HiveServer2Precondition; -import org.apache.hive.kubernetes.operator.dependent.condition.LlapEnabledCondition; -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.TezAmEnabledCondition; +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; @@ -68,54 +52,42 @@ * 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)}) 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: {}/{}", 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)) { + boolean statusChanged = !statusEqualsIgnoringTimestamps(existingStatus, newStatus); + + // Operator-driven autoscaling: evaluate metrics and patch workloads + if (anyAutoscalingEnabled(resource.getSpec())) { + KubernetesClient client = context.getClient(); + HiveClusterAutoscaler scaler = getOrCreateAutoscaler(client); + HiveClusterAutoscaler.AutoscalingEvaluation eval = scaler.evaluate(resource, client); + for (Map.Entry 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(); } @@ -126,8 +98,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(); @@ -172,16 +144,15 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // Metastore status boolean metastoreReady; if (resource.getSpec().metastore().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", - resource.getSpec().metastore().replicas(), - 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", @@ -193,14 +164,15 @@ private HiveClusterStatus buildStatus(HiveCluster resource, } // 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", - resource.getSpec().hiveServer2().replicas(), - 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() && hs2Status.getDesiredReplicas() > 0; + boolean hs2Ready = hs2Status.getReadyReplicas() >= hs2Min; conditions.add(buildCondition("HiveServer2Ready", hs2Ready ? "True" : "False", hs2Ready ? "DeploymentReady" : "DeploymentNotReady", hs2Ready ? "HiveServer2 is ready" : "HiveServer2 not yet ready", @@ -208,19 +180,22 @@ private HiveClusterStatus buildStatus(HiveCluster resource, // LLAP status (optional) if (resource.getSpec().llap().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", - resource.getSpec().llap().replicas(), - 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()) { - status.setTezAm(buildComponentStatus(context, StatefulSet.class, resource.getMetadata().getName() + "-tezam", - resource.getSpec().tezAm().replicas(), - s -> s.getStatus() != null && - s.getStatus().getReadyReplicas() != null ? s.getStatus().getReadyReplicas() : 0)); + 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", + resource.getSpec().tezAm().replicas(), tezAmMin)); } // Overall Ready condition @@ -240,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; } @@ -265,14 +272,157 @@ 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; + } + + 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) { + 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 new file mode 100644 index 000000000000..111a496672da --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java @@ -0,0 +1,261 @@ +/* + * 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.HivePdbDependent; +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(); + + + // 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)); + + + // --- 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/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..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 @@ -73,14 +73,53 @@ 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_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; + 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"; @@ -115,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; + } } 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..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,9 +57,20 @@ 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"); + 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 +106,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()); } @@ -149,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()); @@ -165,6 +191,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()); } 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() {