diff --git a/packaging/src/kubernetes/README.md b/packaging/src/kubernetes/README.md index 1fc11623240c..300eaf835120 100644 --- a/packaging/src/kubernetes/README.md +++ b/packaging/src/kubernetes/README.md @@ -505,19 +505,602 @@ 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: no drain (DAGAppMaster does not expose JMX metrics) + 5. terminationGracePeriodSeconds = gracePeriodSeconds (safety cap) + 6. Pod terminates immediately once drain completes (does NOT wait + the full grace period — it's only the upper safety bound) +``` + +> **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** | `max(ceil(sessions / threshold), cpu_desired)` | Sessions drop to 0 AND CPU below threshold → scale to minReplicas | `hs2_open_sessions`, `jvm_process_cpu_load` | +| **Metastore** | `max(ceil(api_rate / threshold), cpu_desired)` | Rate drops to 0 AND CPU below threshold → scale to minReplicas | `api_*_total`, `jvm_process_cpu_load` | +| **LLAP** | `ceil(avg(queued + configured - available) / scaleUpThreshold)` | All executors idle + no HS2 sessions | `hadoop_llapdaemon_executor*` | +| **Tez AM** | `max(sum(hs2_open_sessions), count(HS2_pods) * sessions_per_queue)` | All HS2 sessions closed | `hs2_open_sessions` (from HS2 pods) | + +**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) | + +### Auto-Suspend (Full Cluster Hibernation) + +Auto-suspend goes beyond scale-to-zero — it fully hibernates the **entire** cluster +(including HS2 and HMS) to 0 replicas after a configurable idle timeout. This is +useful for dev/test clusters that should not consume resources when nobody is using +them. + +**Prerequisites:** Auto-suspend requires autoscaling to be enabled on ALL active +components (HS2, LLAP if enabled, TezAM if enabled, and HMS if `includeMetastore=true`). +The operator will not auto-suspend unless it can confirm all components are at their +minimum state. + +**Idle criteria (all must hold simultaneously for `idleTimeoutMinutes`):** + +| Component | Idle Condition | +|-----------|---------------| +| **HS2** | At `minReplicas` with 0 open sessions | +| **HMS** | At `minReplicas` (only checked if `includeMetastore=true`) | +| **LLAP** | At `minReplicas` (default 0) | +| **TezAM** | At `minReplicas` (default 0) | + +**Important:** HS2 can **only** scale to 0 replicas via auto-suspend. Normal +autoscaling always maintains `minReplicas >= 1` for HS2. Auto-suspend is the +only mechanism that overrides this to achieve full hibernation. + +``` + Auto-Suspend Flow + + 1. Autoscaling scales all components to their minReplicas + (HS2≥1, HMS≥1, LLAP/TezAM to configured min) + + 2. Operator detects idle state: + - HS2 has 0 open sessions + - HMS at minReplicas (if includeMetastore=true) + - LLAP/TezAM at minReplicas + + 3. Idle timer starts (status: clusterPhase=Idle, idleSince=) + + 4. After idleTimeoutMinutes (default 15): + - ALL components scaled to 0 (HMS excluded if includeMetastore=false) + - spec.suspend set to true (cluster stays suspended until user wakes it) + - Status: clusterPhase=Suspended, suspendedSince= + + 5. To wake: kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' + All components restored to minReplicas + (HS2/HMS ≥1, LLAP/TezAM ≥1 for immediate usability) + +``` + +**Configuration:** + +```yaml +cluster: + autoSuspend: + enabled: true + idleTimeoutMinutes: 15 # minutes idle before full hibernation + includeMetastore: true # set false to keep HMS running during suspend +``` + +**Manual Suspend/Wake Commands:** + +```bash +# Suspend immediately (bypasses idle timer) +kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":true}}' + +# Wake cluster (restores to minReplicas) +kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' +``` + +Manual suspend works regardless of whether `autoSuspend.enabled` is true — it +immediately scales all components to 0 without waiting for the idle timeout. +When `includeMetastore: false`, HMS stays running even during manual suspend. + +**Observing cluster state:** + +```bash +# Quick view — printer columns show phase and idle time +kubectl get hivecluster +``` +``` +NAME PHASE IDLE (MIN) AGE +hive Idle 12 2h +``` + +```bash +# After suspend triggers +kubectl get hivecluster +``` +``` +NAME PHASE IDLE (MIN) AGE +hive Suspended 2h +``` + +```bash +# Full status (kubectl get hivecluster hive -o yaml) +``` +```yaml +status: + clusterPhase: Suspended + idleSince: "2026-06-08T10:00:00Z" + idleForMinutes: 15 + suspendedSince: "2026-06-08T10:15:00Z" + conditions: + - type: Suspended + status: "True" + reason: AutoSuspend # or ManualSuspend + message: "Cluster suspended after idle timeout" + lastTransitionTime: "2026-06-08T10:15:00Z" +``` + +When the cluster is running normally: +``` +NAME PHASE IDLE (MIN) AGE +hive Running 2h +``` + +**Full example (autoscaling + auto-suspend):** + +```yaml +cluster: + autoSuspend: + enabled: true + idleTimeoutMinutes: 15 + includeMetastore: false # keep HMS running during suspend + + hiveServer2: + replicas: 10 + autoscaling: + enabled: true + minReplicas: 1 + + metastore: + replicas: 6 + autoscaling: + enabled: true + minReplicas: 1 + + llap: + replicas: 8 + autoscaling: + enabled: true + minReplicas: 0 # scales to 0 via normal autoscaling when HS2 idle + + tezAm: + replicas: 10 + autoscaling: + enabled: true + minReplicas: 0 # scales to 0 via normal autoscaling when HS2 idle +``` + +With this configuration, the cluster lifecycle is: +1. Under load → all components scaled up by autoscaler +2. Load drops → autoscaler scales to minReplicas (HS2=1, HMS=1, LLAP=0, TezAM=0) +3. HS2 idle (0 sessions) for 15 minutes → auto-suspend kicks in → HS2, LLAP, TezAM to 0 (HMS stays at minReplicas) +4. `kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}'` → wake → HS2=1, LLAP=1, TezAM=1 +5. User connects → autoscaler detects sessions → scales up as needed + +### CPU-Based Scaling (HS2 and HMS) + +In addition to the primary metrics (sessions for HS2, API request rate for HMS), +the operator supports a secondary **CPU-based scaling signal** for HiveServer2 and +Metastore. The final desired replica count is: + +``` +final_desired = max(metric_desired, cpu_desired) +``` + +Either signal can trigger scale-up; neither can force scale-down below what the +other recommends. CPU-based scaling uses the same stabilization windows as metric-based +scaling (no separate CPU stabilization). + +**How it works:** + +1. The operator scrapes `ProcessCpuLoad` from `java.lang:type=OperatingSystem` via JMX + Exporter (exported as `jvm_process_cpu_load`, a 0.0–1.0 fraction) +2. Averages across all pods, converts to percentage (0–100) +3. If avg CPU >= `cpuScaleUpThreshold`: scales up proportionally + (`ceil(avgCpu * currentReplicas / cpuScaleUpThreshold)`) +4. If avg CPU < `cpuScaleDownThreshold`: scales down + (`ceil(avgCpu * currentReplicas / cpuScaleUpThreshold)`, floored at `minReplicas`) +5. Between thresholds: holds current replica count + +**Configuration:** + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) that triggers scale-up. Set to `0` to disable CPU-based scaling. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. | + +**Example:** + +```yaml +cluster: + hiveServer2: + replicas: 10 + resources: + limitsCpu: "2" # Recommended: set CPU limits so ProcessCpuLoad is relative to pod allocation + autoscaling: + enabled: true + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 + + metastore: + replicas: 6 + resources: + limitsCpu: "2" + autoscaling: + enabled: true + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 +``` + +**Important: CPU limits and metric accuracy** + +`ProcessCpuLoad` reports CPU usage as a fraction of **available processors**. Without +CPU limits, the JVM sees all node cores (e.g., 8 cores), so even heavy single-pod +load only shows ~12.5%. With `limitsCpu: "2"`, the JVM sees 2 processors and the +metric becomes "% of allocated CPU" — making thresholds meaningful. + +| Pod CPU Limit | JVM sees | 90% threshold means | +|---------------|----------|---------------------| +| None (no limit) | All node cores (e.g., 8) | Using 7.2 of 8 cores — very hard to reach | +| `2` | 2 cores | Using 1.8 of 2 allocated cores | +| `4` | 4 cores | Using 3.6 of 4 allocated cores | + +**Recommendation:** Always set `resources.limitsCpu` when using CPU-based autoscaling. + +**Status output:** + +The operator reports CPU metrics in the HiveCluster status: + +```yaml +status: + hiveServer2: + autoscaling: + currentMetricValue: 5 # total sessions + scaleUpThreshold: 100 + currentCpuPercent: 72.45 # avg ProcessCpuLoad * 100 + cpuScaleUpThreshold: 90 + cpuProposedReplicas: 2 # what CPU alone would recommend + proposedReplicas: 2 + lastScaleTime: "2026-05-31T04:23:07Z" +``` + +**Applicability:** CPU-based scaling only applies to HS2 and HMS. LLAP and TezAM +do not use CPU as a scaling signal (LLAP scales on busy executor slots which already +correlates with CPU; TezAM is demand-based from HS2 session count). + +--- + +### Enabling Autoscaling + +**CLI (with Ozone storage backend):** + +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`, `jvm_process_cpu_load` | Session count for primary scaling + CPU for secondary scaling signal | +| **Metastore** | `api_*_total`, `jvm_process_cpu_load` | API call counters (operator computes request rate from deltas) + CPU for secondary scaling signal | +| **LLAP** | `hadoop_llapdaemon_executornumqueuedrequests`, `hadoop_llapdaemon_executornumexecutorsconfigured`, `hadoop_llapdaemon_executornumexecutorsavailable` | Total busy slots = queued + configured - available | +| **Tez AM** | N/A (scales on HS2 metrics) | TezAM scaling is demand-driven from `hs2_open_sessions` — no TezAM-specific metrics needed | + +### 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. | +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) triggering scale-up. Only HS2/HMS. Set to 0 to disable. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. Only HS2/HMS. | + +--- + ## 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 +1203,28 @@ beeline -u "jdbc:hive2://localhost:10000/" | `cluster.tezAm.extraVolumes` | `[]` | Additional volumes for TezAM pods | | `cluster.tezAm.extraVolumeMounts` | `[]` | Additional volume mounts for TezAM containers | +### Auto-Suspend + +| Value | Default | Description | +|-------|---------|-------------| +| `cluster.autoSuspend.enabled` | `false` | Enable full cluster hibernation after idle timeout. Requires autoscaling enabled on all active components (HMS only if `includeMetastore=true`). | +| `cluster.autoSuspend.idleTimeoutMinutes` | `15` | Minutes of idle time (HS2=0 sessions, LLAP/TezAM at minReplicas) before the cluster suspends. | +| `cluster.autoSuspend.includeMetastore` | `true` | Whether HMS participates in auto-suspend. When false, HMS stays at minReplicas during suspend and HMS autoscaling is not required. | + +### Autoscaling (per component) + +| Value | Default | Description | +|-------|---------|-------------| +| `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` | `100` (HS2/HMS), `10` (LLAP) | Metric threshold per pod triggering scale-up (sessions for HS2, connections for HMS, busy slots for LLAP). TezAM scales 1:1 with demand (no threshold). | +| `cluster..autoscaling.scaleUpStabilizationSeconds` | `60` | Stabilization window for scale-up decisions (prevents flapping) | +| `cluster..autoscaling.scaleDownStabilizationSeconds` | `300-900` | Stabilization window for scale-down decisions (also acts as cooldown between consecutive scale-downs) | +| `cluster..autoscaling.gracePeriodSeconds` | `3600` | Safety cap (seconds) — pod terminates immediately once drain completes, this is only the upper bound | +| `cluster..autoscaling.metricsScrapeIntervalSeconds` | `10` | How often the operator polls JMX metrics from pods. Lower = faster reaction time. | +| `cluster..autoscaling.cpuScaleUpThreshold` | `90` | CPU percentage (0-100) triggering scale-up. Only HS2/HMS. Set to 0 to disable. | +| `cluster..autoscaling.cpuScaleDownThreshold` | `30` | CPU percentage (0-100) below which scale-down is considered. Only HS2/HMS. | + --- ## Upgrade and Uninstall @@ -659,11 +1264,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..fdd8ef33da25 100644 --- a/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml +++ b/packaging/src/kubernetes/helm/hive-operator/crds/hiveclusters.hive.apache.org-v1.yml @@ -13,12 +13,47 @@ spec: singular: hivecluster scope: Namespaced versions: - - name: v1 + - additionalPrinterColumns: + - jsonPath: .status.clusterPhase + name: Phase + priority: 0 + type: string + - jsonPath: .status.idleForMinutes + name: Idle (min) + priority: 0 + type: integer + - jsonPath: .status.suspendedSince + name: Suspended Since + priority: 1 + type: string + name: v1 schema: openAPIV3Schema: properties: spec: properties: + autoSuspend: + description: "Auto-suspend configuration. When enabled and all components\ + \ are idle for the configured timeout, the cluster scales to 0 replicas." + properties: + enabled: + default: false + description: "Whether auto-suspend is enabled. Requires autoscaling\ + \ to be enabled on all active components (HS2, LLAP if enabled,\ + \ TezAM if enabled, and HMS if includeMetastore is true)." + type: boolean + idleTimeoutMinutes: + default: 15 + description: "Minutes of idle time (HS2=0 sessions, LLAP/TezAM\ + \ at minReplicas) before the cluster auto-suspends." + type: integer + includeMetastore: + default: true + description: "Whether Metastore participates in auto-suspend.\ + \ When false, HMS stays at minReplicas during suspend and HMS\ + \ autoscaling is not required for auto-suspend to activate." + type: boolean + type: object envVars: description: "Environment variables injected into all component pods\ \ (e.g., storage credentials, custom JVM options)" @@ -44,6 +79,70 @@ spec: hiveServer2: description: HiveServer2 component configuration properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + 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 + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: How often (seconds) the operator scrapes JMX + 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 +251,70 @@ spec: llap: description: LLAP daemon configuration. Enabled by default. properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + 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 + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: How often (seconds) the operator scrapes JMX + 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 +398,70 @@ spec: metastore: description: Metastore component configuration properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + 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 + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: How often (seconds) the operator scrapes JMX + 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 @@ -368,9 +595,77 @@ spec: type: string type: object x-kubernetes-preserve-unknown-fields: true + suspend: + description: "When true, the cluster is immediately suspended (all\ + \ components scaled to 0). Set to false to wake a suspended cluster." + type: boolean tezAm: description: Tez Application Master configuration. Enabled by default. properties: + autoscaling: + description: "Autoscaling configuration (operator-driven, no external\ + \ dependencies)" + properties: + cpuScaleDownThreshold: + default: 30 + description: CPU percentage (0-100) below which scale-down + is considered. Only applies to HS2 and HMS. + type: integer + cpuScaleUpThreshold: + default: 90 + description: CPU percentage (0-100) that triggers scale-up. + Only applies to HS2 and HMS. Set to 0 to disable CPU-based + scaling. + type: integer + enabled: + default: false + description: Whether autoscaling is enabled for this component + 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 + metricsPort: + default: 9404 + description: Port on which the Prometheus JMX Exporter serves + metrics. The operator scrapes this port on each pod for + autoscaling decisions. + type: integer + metricsScrapeIntervalSeconds: + default: 10 + description: How often (seconds) the operator scrapes JMX + 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 @@ -457,6 +752,8 @@ spec: x-kubernetes-preserve-unknown-fields: true status: properties: + clusterPhase: + type: string conditions: items: properties: @@ -476,16 +773,62 @@ spec: type: array hiveServer2: properties: - desiredReplicas: + autoscaling: + properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number + 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 readyReplicas: type: integer type: object + idleForMinutes: + type: integer + idleSince: + type: string llap: properties: - desiredReplicas: + autoscaling: + properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number + 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 +837,28 @@ spec: type: object metastore: properties: - desiredReplicas: + autoscaling: + properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number + 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 @@ -503,9 +867,32 @@ spec: type: object observedGeneration: type: integer + suspendedSince: + type: string tezAm: properties: - desiredReplicas: + autoscaling: + properties: + cpuProposedReplicas: + type: integer + cpuScaleUpThreshold: + type: integer + currentCpuPercent: + type: number + 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..6b053ad4ec3b 100644 --- a/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/templates/hivecluster.yaml @@ -67,6 +67,18 @@ 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 }} + cpuScaleUpThreshold: {{ .Values.cluster.metastore.autoscaling.cpuScaleUpThreshold | default 90 }} + cpuScaleDownThreshold: {{ .Values.cluster.metastore.autoscaling.cpuScaleDownThreshold | default 30 }} + {{- end }} {{- else }} {{- if .Values.cluster.metastore.externalUri }} externalUri: {{ .Values.cluster.metastore.externalUri | quote }} @@ -96,6 +108,18 @@ 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 }} + cpuScaleUpThreshold: {{ .Values.cluster.hiveServer2.autoscaling.cpuScaleUpThreshold | default 90 }} + cpuScaleDownThreshold: {{ .Values.cluster.hiveServer2.autoscaling.cpuScaleDownThreshold | default 30 }} + {{- end }} llap: enabled: {{ .Values.cluster.llap.enabled }} @@ -120,6 +144,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 +180,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: @@ -176,4 +219,15 @@ spec: volumeMounts: {{- toYaml .Values.cluster.storage.volumeMounts | nindent 4 }} {{- end }} + + {{- if and .Values.cluster.autoSuspend .Values.cluster.autoSuspend.enabled }} + autoSuspend: + enabled: true + idleTimeoutMinutes: {{ .Values.cluster.autoSuspend.idleTimeoutMinutes | default 15 }} + {{- if hasKey .Values.cluster.autoSuspend "includeMetastore" }} + includeMetastore: {{ .Values.cluster.autoSuspend.includeMetastore }} + {{- end }} + {{- end }} + + suspend: false {{- end }} diff --git a/packaging/src/kubernetes/helm/hive-operator/values.yaml b/packaging/src/kubernetes/helm/hive-operator/values.yaml index b7d75930c5b2..99e9b47b6a17 100644 --- a/packaging/src/kubernetes/helm/hive-operator/values.yaml +++ b/packaging/src/kubernetes/helm/hive-operator/values.yaml @@ -101,6 +101,20 @@ cluster: # mountPath: /etc/gcs # readOnly: true + # --------------------------------------------------------------------------- + # AUTO-SUSPEND — fully hibernates the cluster after idle timeout + # --------------------------------------------------------------------------- + # When enabled (requires autoscaling on all active components), the operator + # scales the entire cluster to 0 replicas after all components have been idle + # for idleTimeoutMinutes. Use kubectl patch to manually suspend/wake: + # kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":true}}' + # kubectl patch hivecluster hive --type=merge -p '{"spec":{"suspend":false}}' + autoSuspend: + enabled: false + idleTimeoutMinutes: 15 + # Set to false to keep HMS running during suspend (HMS autoscaling not required) + includeMetastore: true + # --------------------------------------------------------------------------- # METASTORE — defaults to enabled, 2 replicas (HA) # --------------------------------------------------------------------------- @@ -112,6 +126,19 @@ 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: 100 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 300 + gracePeriodSeconds: 60 + metricsScrapeIntervalSeconds: 10 + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 # Set to use an external Metastore instead of deploying one: # enabled: false # externalUri: "thrift://external-metastore:9083" @@ -127,6 +154,18 @@ 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: 100 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 600 + gracePeriodSeconds: 300 + metricsScrapeIntervalSeconds: 10 + cpuScaleUpThreshold: 90 + cpuScaleDownThreshold: 30 # --------------------------------------------------------------------------- # LLAP — enabled by default for full-HA @@ -141,6 +180,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: 10 + scaleUpStabilizationSeconds: 60 + scaleDownStabilizationSeconds: 900 + gracePeriodSeconds: 600 + metricsScrapeIntervalSeconds: 10 # --------------------------------------------------------------------------- # TEZ AM — enabled by default for full-HA @@ -154,3 +204,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/pom.xml b/packaging/src/kubernetes/pom.xml index 342441730cc5..6cb6d6ce0c24 100644 --- a/packaging/src/kubernetes/pom.xml +++ b/packaging/src/kubernetes/pom.xml @@ -65,9 +65,14 @@ ${fabric8.version} provided + + org.slf4j + slf4j-api + 2.0.16 + org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl ${log4j2.version} @@ -189,6 +194,7 @@ docker build + --no-cache -t apache/hive:operator-${project.version} . 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/BackgroundMetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java new file mode 100644 index 000000000000..26c2fc83d668 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/BackgroundMetricsScraper.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.kubernetes.operator.autoscaling; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Runs periodic metrics scraping in the background so that the JOSDK reconcile + * thread is never blocked by HTTP calls to pod JMX exporters. + *

+ * Each component gets its own scheduled task that writes results to a shared + * {@link MetricsCache}. The reconciler reads from that cache (non-blocking). + */ +public class BackgroundMetricsScraper { + + private static final Logger LOG = LoggerFactory.getLogger(BackgroundMetricsScraper.class); + + private final ScheduledExecutorService scheduler; + private final MetricsScraper scraper; + private final MetricsCache cache; + // Key: "namespace/clusterName/component" → active scrape task + private final ConcurrentHashMap> activeTasks = + new ConcurrentHashMap<>(); + // Tracks registered intervals to detect spec changes + private final ConcurrentHashMap registeredIntervals = + new ConcurrentHashMap<>(); + + public BackgroundMetricsScraper(MetricsScraper scraper, MetricsCache cache) { + this.scraper = scraper; + this.cache = cache; + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + Thread t = new Thread(r, "hive-metrics-scraper"); + t.setDaemon(true); + return t; + }); + } + + /** + * Registers (or updates) a periodic scrape task for a component. + * Idempotent — only recreates the task if the interval has changed. + * + * @param namespace the Kubernetes namespace + * @param clusterName the HiveCluster name + * @param component the component name (e.g., "hiveserver2") + * @param selector label selector for pod listing + * @param metricsPort the JMX exporter port + * @param intervalSecs how often to scrape (from AutoscalingSpec) + */ + public void registerOrUpdate(String namespace, String clusterName, + String component, Map selector, + int metricsPort, int intervalSecs) { + String key = namespace + "/" + clusterName + "/" + component; + Integer existing = registeredIntervals.get(key); + if (existing != null && existing == intervalSecs) { + return; // Already registered with same interval + } + + // Cancel existing task if interval changed + ScheduledFuture oldTask = activeTasks.remove(key); + if (oldTask != null) { + oldTask.cancel(false); + } + + ScheduledFuture future = scheduler.scheduleWithFixedDelay( + () -> scrapeAndStore(key, namespace, selector, metricsPort), + 0, intervalSecs, TimeUnit.SECONDS); + + activeTasks.put(key, future); + registeredIntervals.put(key, intervalSecs); + LOG.debug("Registered background scrape for {} (interval={}s)", key, intervalSecs); + } + + /** + * Unregisters all scrape tasks for a deleted cluster. + */ + public void unregisterCluster(String namespace, String clusterName) { + String prefix = namespace + "/" + clusterName + "/"; + activeTasks.entrySet().removeIf(entry -> { + if (entry.getKey().startsWith(prefix)) { + entry.getValue().cancel(false); + return true; + } + return false; + }); + registeredIntervals.keySet().removeIf(k -> k.startsWith(prefix)); + cache.removeByPrefix(prefix); + LOG.debug("Unregistered background scrape tasks for {}/{}", namespace, clusterName); + } + + /** + * Shuts down the background scheduler. Called on operator shutdown. + */ + public void shutdown() { + scheduler.shutdownNow(); + } + + private void scrapeAndStore(String key, String namespace, + Map selector, int metricsPort) { + try { + List metrics = scraper.scrape(namespace, selector, metricsPort); + cache.put(key, metrics); + } catch (Exception e) { + // Do not update cache on failure — staleness check handles it + LOG.debug("Background scrape failed for {}: {}", key, e.getMessage()); + } + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.java new file mode 100644 index 000000000000..6824b0e8cb79 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/ComponentAutoscaler.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.autoscaling; + +import java.time.Duration; +import java.util.List; + +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Per-component autoscaler state. Owns the scaling strategy, + * stabilization windows. + */ +public class ComponentAutoscaler { + + /** Result of an autoscaling evaluation. */ + public record EvaluationResult(int rawMetricValue, double cpuPercent, + int cpuProposedReplicas, int proposedReplicas, Integer patchTo) {} + + + private static final Logger LOG = LoggerFactory.getLogger(ComponentAutoscaler.class); + + private static final String METRIC_CPU_LOAD = "jvm_process_cpu_load"; + + private final String component; + private final ScalingStrategy strategy; + private final boolean cpuScalingApplicable; + private StabilizationWindow scaleUpWindow; + private StabilizationWindow scaleDownWindow; + private int lastScaleUpStabilization = -1; + private int lastScaleDownStabilization = -1; + private boolean initialized; + private double lastCpuPercent; + + public ComponentAutoscaler(String component, ScalingStrategy strategy) { + this.component = component; + this.strategy = strategy; + this.cpuScalingApplicable = ConfigUtils.COMPONENT_HIVESERVER2.equals(component) + || ConfigUtils.COMPONENT_METASTORE.equals(component); + } + + /** 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(); + + // CPU-based scaling: combine with metric-based desired via max() + int cpuDesired = computeCpuDesired(metrics, spec, currentReplicas); + int combined = Math.max(rawDesired, cpuDesired); + int clamped = Math.max(spec.minReplicas(), Math.min(combined, maxReplicas)); + + scaleUpWindow.record(clamped); + scaleDownWindow.record(clamped); + + 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, lastCpuPercent, cpuDesired, clamped, null); + } + + if (target < currentReplicas) { + LOG.info("[{}] Scaling down: {} -> {}", component, currentReplicas, target); + } else { + LOG.info("[{}] Scaling up: {} -> {}", component, currentReplicas, target); + } + return new EvaluationResult(metricValue, lastCpuPercent, cpuDesired, clamped, target); + } + + /** + * Compute desired replicas based on CPU utilization. + * Returns 0 if CPU scaling is not applicable or no CPU data is available. + */ + private int computeCpuDesired(List metrics, AutoscalingSpec spec, int currentReplicas) { + if (!cpuScalingApplicable || spec.cpuScaleUpThreshold() <= 0 || metrics.isEmpty()) { + lastCpuPercent = 0; + return 0; + } + + double totalCpu = 0; + int count = 0; + for (PodMetrics pm : metrics) { + Double cpu = pm.metrics().get(METRIC_CPU_LOAD); + if (cpu != null) { + totalCpu += cpu * 100.0; + count++; + } + } + if (count == 0) { + lastCpuPercent = 0; + return 0; + } + double avgCpuPercent = totalCpu / count; + lastCpuPercent = avgCpuPercent; + LOG.debug("[{}] CPU raw: totalCpu={}, count={}, avg={}%", component, totalCpu, count, avgCpuPercent); + + if (avgCpuPercent >= spec.cpuScaleUpThreshold()) { + // Scale up proportionally: how many pods to bring avg below threshold + return (int) Math.ceil(avgCpuPercent * currentReplicas / spec.cpuScaleUpThreshold()); + } else if (avgCpuPercent < spec.cpuScaleDownThreshold()) { + // Scale down: current load could fit in fewer pods + int desired = (int) Math.ceil(avgCpuPercent * currentReplicas / spec.cpuScaleUpThreshold()); + return Math.max(desired, spec.minReplicas()); + } + // Between thresholds: hold current + return currentReplicas; + } + + private void ensureWindows(AutoscalingSpec spec) { + 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..b091f397c485 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/HiveClusterAutoscaler.java @@ -0,0 +1,353 @@ +/* + * 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.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.ConfigUtils; +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); + } + + /** + * Sets the managed replica count for a component. Used by suspend/wake logic + * to override what the autoscaler would normally compute. + */ + public static void setManagedReplicas(String namespace, String clusterName, + String component, int replicas) { + MANAGED_REPLICAS.put(namespace + "/" + clusterName + "/" + component, replicas); + } + + private record PendingScaleDown(int targetReplicas, Instant annotatedAt) {} + + private final MetricsScraper scraper; + private final BackgroundMetricsScraper bgScraper; + private final MetricsCache metricsCache; + // Key: "namespace/clusterName/component" + private final ConcurrentHashMap autoscalers = + new ConcurrentHashMap<>(); + private final ConcurrentHashMap lastScaleTimes = + new ConcurrentHashMap<>(); + // Two-phase scale-down: holds deferred scale-down targets while pod-deletion-cost + // annotations propagate (2s delay before applying the actual scale patch). + private final ConcurrentHashMap pendingScaleDowns = + new ConcurrentHashMap<>(); + + public HiveClusterAutoscaler(MetricsScraper scraper, + BackgroundMetricsScraper bgScraper, MetricsCache metricsCache) { + this.scraper = scraper; + this.bgScraper = bgScraper; + this.metricsCache = metricsCache; + } + + public BackgroundMetricsScraper getBackgroundScraper() { + return bgScraper; + } + + /** + * Removes all in-memory state for a deleted HiveCluster to prevent memory leaks. + */ + public void cleanupCluster(String namespace, String clusterName) { + String prefix = namespace + "/" + clusterName + "/"; + MANAGED_REPLICAS.keySet().removeIf(k -> k.startsWith(prefix)); + autoscalers.keySet().removeIf(k -> k.startsWith(prefix)); + lastScaleTimes.keySet().removeIf(k -> k.startsWith(prefix)); + pendingScaleDowns.keySet().removeIf(k -> k.startsWith(prefix)); + LOG.info("Cleaned up autoscaler state for {}/{}", namespace, clusterName); + } + + /** + * Returns true if there are pending scale-down operations waiting for + * annotation propagation. The reconciler should reschedule sooner (2s) + * when this returns true. + */ + public boolean hasPendingScaleDowns() { + return !pendingScaleDowns.isEmpty(); + } + + /** + * 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()) { + AutoscalingSpec hs2Auto = spec.hiveServer2().autoscaling(); + String hs2Key = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_HIVESERVER2; + Map hs2Selector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_HIVESERVER2); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Selector, + hs2Auto.metricsPort(), hs2Auto.metricsScrapeIntervalSeconds()); + int maxStale = hs2Auto.metricsScrapeIntervalSeconds() * 3; + List hs2Metrics = metricsCache.getOrEmpty(hs2Key, maxStale); + + // Two-phase scale-down: check if a pending scale-down from a prior + // reconcile is ready to be applied (annotations have propagated). + PendingScaleDown pending = pendingScaleDowns.get(hs2Key); + if (pending != null) { + if (Duration.between(pending.annotatedAt(), Instant.now()).toSeconds() >= 2) { + patches.put(ConfigUtils.COMPONENT_HIVESERVER2, pending.targetReplicas()); + MANAGED_REPLICAS.put(hs2Key, pending.targetReplicas()); + lastScaleTimes.put(hs2Key, Instant.now().toString()); + pendingScaleDowns.remove(hs2Key); + LOG.info("[hiveserver2] Applying deferred scale-down to {} replicas", pending.targetReplicas()); + } + // Build status even when waiting for pending scale-down + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, + spec.hiveServer2().replicas(), new HashMap<>(), statuses, hs2Metrics); + } else { + // Pod deletion cost only applies to Deployments (ReplicaSet controller). + // StatefulSets always scale down by highest ordinal regardless of this + // annotation. LLAP/TezAM graceful drain is handled by preStop hooks. + updateDeploymentPodDeletionCost(client, namespace, hs2Metrics, "hs2_open_sessions"); + + Map hs2Patches = new HashMap<>(); + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_HIVESERVER2, hs2Auto, + spec.hiveServer2().replicas(), hs2Patches, statuses, hs2Metrics); + + Integer hs2Patch = hs2Patches.get(ConfigUtils.COMPONENT_HIVESERVER2); + int currentReplicas = getCurrentReplicas(client, namespace, clusterName, ConfigUtils.COMPONENT_HIVESERVER2); + if (hs2Patch != null && hs2Patch < currentReplicas) { + // Scale-down: defer to allow deletion-cost annotations to propagate + pendingScaleDowns.put(hs2Key, new PendingScaleDown(hs2Patch, Instant.now())); + LOG.info("[hiveserver2] Deferring scale-down to {} (waiting for deletion-cost propagation)", + hs2Patch); + } else if (hs2Patch != null) { + // Scale-up: apply immediately + patches.put(ConfigUtils.COMPONENT_HIVESERVER2, hs2Patch); + MANAGED_REPLICAS.put(hs2Key, hs2Patch); + } + } + } + + // Metastore + if (spec.metastore().isEnabled() && spec.metastore().autoscaling().isEnabled()) { + AutoscalingSpec msAuto = spec.metastore().autoscaling(); + Map msSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_METASTORE); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_METASTORE, msSelector, + msAuto.metricsPort(), msAuto.metricsScrapeIntervalSeconds()); + String msKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_METASTORE; + List msMetrics = metricsCache.getOrEmpty(msKey, msAuto.metricsScrapeIntervalSeconds() * 3); + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_METASTORE, msAuto, + spec.metastore().replicas(), patches, statuses, msMetrics); + } + + // LLAP + if (spec.llap().isEnabled() && spec.llap().autoscaling().isEnabled()) { + AutoscalingSpec llapAuto = spec.llap().autoscaling(); + Map llapSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_LLAP); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_LLAP, llapSelector, + llapAuto.metricsPort(), llapAuto.metricsScrapeIntervalSeconds()); + String llapKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_LLAP; + List llapMetrics = metricsCache.getOrEmpty(llapKey, llapAuto.metricsScrapeIntervalSeconds() * 3); + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_LLAP, llapAuto, + spec.llap().replicas(), patches, statuses, llapMetrics); + } + + // TezAM + if (spec.tezAm().isEnabled() && spec.tezAm().autoscaling().isEnabled()) { + AutoscalingSpec tezAuto = spec.tezAm().autoscaling(); + Map tezSelector = Labels.selectorForComponent(cluster, ConfigUtils.COMPONENT_TEZAM); + bgScraper.registerOrUpdate(namespace, clusterName, + ConfigUtils.COMPONENT_TEZAM, tezSelector, + tezAuto.metricsPort(), tezAuto.metricsScrapeIntervalSeconds()); + String tezKey = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_TEZAM; + List tezMetrics = metricsCache.getOrEmpty(tezKey, tezAuto.metricsScrapeIntervalSeconds() * 3); + evaluateComponent(cluster, client, namespace, clusterName, + ConfigUtils.COMPONENT_TEZAM, tezAuto, + spec.tezAm().replicas(), patches, statuses, tezMetrics); + } + + return new AutoscalingEvaluation(patches, statuses); + } + + /** + * Returns cached HS2 metrics (used by LLAP/TezAM activation gate). + * Non-blocking — reads from the background-scraper cache. + */ + public List getHs2MetricsFromCache(HiveCluster cluster) { + String namespace = cluster.getMetadata().getNamespace(); + String clusterName = cluster.getMetadata().getName(); + String key = namespace + "/" + clusterName + "/" + ConfigUtils.COMPONENT_HIVESERVER2; + int maxStale = cluster.getSpec().hiveServer2().autoscaling().metricsScrapeIntervalSeconds() * 3; + return metricsCache.getOrEmpty(key, maxStale); + } + + private void evaluateComponent(HiveCluster cluster, KubernetesClient client, + String namespace, String clusterName, String component, + AutoscalingSpec autoscaling, int maxReplicas, + Map patches, Map statuses, + List metrics) { + + int currentReplicas = getCurrentReplicas(client, namespace, clusterName, component); + + String key = namespace + "/" + clusterName + "/" + component; + + // For LLAP and TezAM, scaling decisions are based on HS2 metrics (activation gate), + // not their own pod metrics. Allow evaluation even with 0 own pods. + boolean usesHs2Activation = ConfigUtils.COMPONENT_LLAP.equals(component) + || ConfigUtils.COMPONENT_TEZAM.equals(component); + + if (metrics.isEmpty() && !usesHs2Activation) { + LOG.debug("[{}] No ready pods to scrape, skipping", component); + 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()); + } + // CPU metrics (only for HS2 and HMS — LLAP/TezAM don't use CPU-based scaling) + if ((ConfigUtils.COMPONENT_HIVESERVER2.equals(component) || ConfigUtils.COMPONENT_METASTORE.equals(component)) + && autoscaling.cpuScaleUpThreshold() > 0) { + as.setCurrentCpuPercent(result.cpuPercent()); + as.setCpuScaleUpThreshold(autoscaling.cpuScaleUpThreshold()); + as.setCpuProposedReplicas(result.cpuProposedReplicas()); + } + as.setProposedReplicas(result.proposedReplicas()); + as.setLastScaleTime(lastScaleTimes.get(key)); + statuses.put(component, as); + + 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 ConfigUtils.COMPONENT_HIVESERVER2 -> new HiveServer2ScalingStrategy(); + case ConfigUtils.COMPONENT_METASTORE -> new MetastoreScalingStrategy(); + case ConfigUtils.COMPONENT_LLAP -> new LlapScalingStrategy(this, cluster); + case ConfigUtils.COMPONENT_TEZAM -> new TezAmScalingStrategy(this, cluster); + default -> throw new IllegalArgumentException("Unknown component: " + component); + }; + } + + private int getCurrentReplicas(KubernetesClient client, String namespace, + String clusterName, String component) { + String workloadName = clusterName + "-" + component; + if (ConfigUtils.COMPONENT_LLAP.equals(component) || ConfigUtils.COMPONENT_TEZAM.equals(component)) { + var ss = client.apps().statefulSets() + .inNamespace(namespace).withName(workloadName).get(); + return ss != null && ss.getSpec().getReplicas() != null ? ss.getSpec().getReplicas() : 0; + } 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). + *

+ * Only meaningful for Deployments (HS2, Metastore) — the ReplicaSet controller + * respects this annotation. StatefulSets ignore it and always terminate by ordinal. + */ + private void updateDeploymentPodDeletionCost(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..e530f5adae66 --- /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.getHs2MetricsFromCache(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/MetricsCache.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java new file mode 100644 index 000000000000..75b230ef2d64 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsCache.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.kubernetes.operator.autoscaling; + +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Thread-safe cache for scraped Prometheus metrics from pods. + * Entries become stale after a configurable duration (typically 3x the scrape interval) + * and are treated as absent when read. + */ +public class MetricsCache { + + private record CachedResult(List metrics, Instant scrapedAt) {} + + private final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + + /** + * Stores scraped metrics for a component. + * + * @param key format: "namespace/clusterName/component" + * @param metrics the scraped pod metrics + */ + public void put(String key, List metrics) { + cache.put(key, new CachedResult(metrics, Instant.now())); + } + + /** + * Returns cached metrics if present and not stale, otherwise an empty list. + * + * @param key format: "namespace/clusterName/component" + * @param maxStaleSecs maximum age in seconds before the entry is considered stale + * @return the cached metrics, or an empty list if absent or stale + */ + public List getOrEmpty(String key, int maxStaleSecs) { + CachedResult result = cache.get(key); + if (result == null) { + return Collections.emptyList(); + } + if (Instant.now().isAfter(result.scrapedAt().plusSeconds(maxStaleSecs))) { + return Collections.emptyList(); + } + return result.metrics(); + } + + /** + * Removes all entries whose key starts with the given prefix. + * Used for cleanup when a HiveCluster is deleted. + */ + public void removeByPrefix(String prefix) { + cache.keySet().removeIf(k -> k.startsWith(prefix)); + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/MetricsScraper.java new file mode 100644 index 000000000000..c3aa8aa82ca8 --- /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 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) + * @param metricsPort the port on which the Prometheus JMX Exporter serves metrics + * @return list of per-pod metrics (empty if no pods or all fail) + */ + public List scrape(String namespace, Map selector, int metricsPort) { + 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, metricsPort) + .thenApply(body -> new PodMetrics(podName, PrometheusTextParser.parse(body))) + .exceptionally(ex -> { + LOG.debug("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, int metricsPort) { + URI uri = URI.create("http://" + podIp + ":" + metricsPort + "/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..8940a80b1934 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/PrometheusTextParser.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.kubernetes.operator.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. + * Labels are stripped from keys; duplicate metric names (from multiple + * label sets) are summed. + */ + public static Map parse(String body) { + return doParse(body, false); + } + + /** + * Parse and return per-label-set metrics (preserving labels in key). + * Key format: "metric_name{label=value,...}" + */ + public static Map parseWithLabels(String body) { + return doParse(body, true); + } + + private static Map doParse(String body, boolean keepLabels) { + Map result = new HashMap<>(); + if (body == null || body.isEmpty()) { + return result; + } + 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 = keepLabels ? line.substring(0, braceEnd + 1) : 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); + if (keepLabels) { + result.put(metricKey, value); + } else { + 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; + } +} 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..d2863102b098 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/autoscaling/TezAmScalingStrategy.java @@ -0,0 +1,93 @@ +/* + * 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 Tez Application Master. + * TezAM scaling tracks HS2 session demand: desired = ceil(sum(hs2_open_sessions)). + *

+ * 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.getHs2MetricsFromCache(cluster); + + // Activation gate: if HS2 scrape returns no data but TezAM has running pods, + // treat as "unknown" and preserve current state to avoid spurious scale-to-zero. + 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; + for (PodMetrics pm : hs2Metrics) { + totalSessions += pm.metrics().getOrDefault( + HiveServer2ScalingStrategy.METRIC_OPEN_SESSIONS, 0.0); + } + + if (totalSessions <= 0) { + LOG.debug("[tezam] No HS2 sessions, scaling to minReplicas"); + lastMetric = 0; + return autoscaling.minReplicas(); + } + + lastMetric = (int) totalSessions; + + // Scale based on concurrent demand — one TezAM per open HS2 session + int desired = (int) Math.ceil(totalSessions); + desired = Math.min(desired, maxReplicas); + + LOG.debug("[tezam] totalSessions={}, desired={}", totalSessions, 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..2ca7f87232dd --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveConfigMapDependent.java @@ -0,0 +1,154 @@ +/* + * 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.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; +import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * 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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_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..9f7a345daffd 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveDependentResource.java @@ -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,37 @@ 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(); + } + String expectedName = getSecondaryResourceName(primary, + context); + return resources.stream() + .filter(r -> expectedName.equals( + r.getMetadata().getName())) + .findFirst(); + }); } /** @@ -125,6 +134,208 @@ public Matcher.Result match(R actualResource, R desired, return super.match(actualResource, desired, primary, context); } + @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) { + // Suspended cluster → 0 replicas (dependent resources natively respect suspend). + // Exception: HMS stays running if includeMetastore=false in autoSuspend config. + if (primary instanceof HiveCluster hc && hc.getSpec().suspend()) { + boolean isMetastore = ConfigUtils.COMPONENT_METASTORE.equals(getComponentName()); + if (!isMetastore || hc.getSpec().autoSuspend().includeMetastore()) { + return 0; + } + } + 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 +446,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 +633,233 @@ 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, int metricsPort, + 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(metricsPort) + .withProtocol("TCP").build()); + + // Add javaagent flag to the appropriate JVM opts env var. + // LLAP uses LLAP_DAEMON_OPTS (its startup script ignores SERVICE_OPTS). + String agentArg = String.format("-javaagent:%s=%d:%s", + JMX_EXPORTER_JAR, metricsPort, JMX_EXPORTER_CONFIG); + String optsEnvVar = ConfigUtils.COMPONENT_LLAP.equals(component) ? "LLAP_DAEMON_OPTS" : "SERVICE_OPTS"; + boolean found = false; + for (int i = 0; i < envVars.size(); i++) { + 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 ConfigUtils.COMPONENT_HIVESERVER2: + // HS2 session and operation metrics + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: hs2_$1\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: hs2_active_calls_$1\n"); + sb.append(" type: GAUGE\n"); + // Tez session pool metrics (pending tasks, backlog ratio, running tasks) + sb.append("- pattern: 'metrics<>Value'\n"); + sb.append(" name: tez_session_$1\n"); + sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_METASTORE: + // HMS API call metrics + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: api_$1_total\n"); + sb.append(" type: COUNTER\n"); + sb.append("- pattern: 'metrics<>Count'\n"); + sb.append(" name: hive_metastore_open_connections\n"); + sb.append(" type: GAUGE\n"); + // JVM CPU usage for CPU-based autoscaling + sb.append("- pattern: 'java.lang<>ProcessCpuLoad'\n"); + sb.append(" name: jvm_process_cpu_load\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_LLAP: + // Only export the executor metrics the autoscaler and drain script need. + // A wildcard '.*' pattern serializes 600+ metrics every scrape interval, + // causing CPU spikes and GC pressure on the LLAP JVM. + // Internal format: Hadoop><>Attribute + // Separate rules per attribute — JMX Exporter 1.x caches per-bean, not per-attribute. + sb.append("- pattern: 'Hadoop<>ExecutorNumQueuedRequests'\n"); + sb.append(" name: hadoop_llapdaemon_executornumqueuedrequests\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsConfigured'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsconfigured\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutorsAvailable'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutorsavailable\n"); + sb.append(" type: GAUGE\n"); + sb.append("- pattern: 'Hadoop<>ExecutorNumExecutors'\n"); + sb.append(" name: hadoop_llapdaemon_executornumexecutors\n"); + sb.append(" type: GAUGE\n"); + break; + case ConfigUtils.COMPONENT_TEZAM: + // TezAM DAG execution metrics + sb.append("- pattern: 'Hadoop<>(.+)'\n"); + sb.append(" name: tez_am_$1\n"); + sb.append(" type: GAUGE\n"); + break; + default: + sb.append("- pattern: '.*'\n"); + break; + } + return sb.toString(); + } + } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java new file mode 100644 index 000000000000..5a3e6cdc493d --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HivePdbDependent.java @@ -0,0 +1,112 @@ +/* + * 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.ConfigUtils; +import org.apache.hive.kubernetes.operator.util.Labels; + +/** + * Unified PodDisruptionBudget dependent resource for all Hive components. + * Uses maxUnavailable=1 to allow at most one pod to be disrupted at a time + * while still permitting node drains when replicas=1. + *

+ * Subclassed per component (HS2, Metastore, LLAP, TezAM) only to satisfy + * JOSDK's requirement for distinct no-arg-constructible classes in the workflow. + */ +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() + .withMaxUnavailable(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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_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..bf18ed7a7236 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServer2DeploymentDependent.java @@ -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; @@ -48,13 +49,24 @@ public class HiveServer2DeploymentDependent extends HiveDependentResource { - public static final String COMPONENT = "hiveserver2"; + public static final String COMPONENT = ConfigUtils.COMPONENT_HIVESERVER2; private static final String SCRATCH_MOUNT_PATH = "/opt/hive/scratch"; public HiveServer2DeploymentDependent() { 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) { @@ -64,7 +76,7 @@ protected Deployment desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "hiveserver2", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("TEZ_AM_EXTERNAL_ID", "tez-session-hs2", null)); @@ -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, autoscaling.metricsPort(), + 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,20 +233,20 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(hs2.replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "hiveserver2") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("hiveserver2") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) @@ -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..e367bf4c6028 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/HiveServiceDependent.java @@ -0,0 +1,186 @@ +/* + * 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(ConfigUtils.COMPONENT_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(ConfigUtils.COMPONENT_METASTORE); + } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + var overrides = hiveCluster.getSpec().metastore().configOverrides(); + int thriftPort = ConfigUtils.getInt(overrides, + ConfigUtils.METASTORE_THRIFT_PORT_KEY, + ConfigUtils.METASTORE_THRIFT_PORT_HIVE_KEY, + ConfigUtils.METASTORE_THRIFT_PORT_DEFAULT); + int restPort = ConfigUtils.getInt(overrides, + ConfigUtils.METASTORE_REST_HTTP_PORT_KEY, + null, ConfigUtils.METASTORE_REST_HTTP_PORT_DEFAULT); + builder.editSpec() + .withType("ClusterIP") + .addNewPort().withName("thrift").withProtocol("TCP") + .withPort(thriftPort).withTargetPort(new IntOrString(thriftPort)).endPort() + .addNewPort().withName("rest").withProtocol("TCP") + .withPort(restPort).withTargetPort(new IntOrString(restPort)).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(ConfigUtils.COMPONENT_LLAP); + } + + @Override + protected void customizeSpec(ServiceBuilder builder, HiveCluster hiveCluster) { + var overrides = hiveCluster.getSpec().llap().configOverrides(); + int managementPort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT); + int shufflePort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT); + int webPort = ConfigUtils.getInt(overrides, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT); + builder.editSpec() + .withClusterIP("None") + .addNewPort().withName("management").withProtocol("TCP") + .withPort(managementPort).withTargetPort(new IntOrString(managementPort)).endPort() + .addNewPort().withName("shuffle").withProtocol("TCP") + .withPort(shufflePort).withTargetPort(new IntOrString(shufflePort)).endPort() + .addNewPort().withName("web").withProtocol("TCP") + .withPort(webPort).withTargetPort(new IntOrString(webPort)).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(ConfigUtils.COMPONENT_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..380a7bea7967 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/LlapStatefulSetDependent.java @@ -34,7 +34,9 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -50,12 +52,23 @@ public class LlapStatefulSetDependent extends HiveDependentResource { - public static final String COMPONENT = "llap"; + public static final String COMPONENT = ConfigUtils.COMPONENT_LLAP; 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) { @@ -65,7 +78,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "llap", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("LLAP_MEMORY_MB", String.valueOf(llap.memoryMb()), null)); @@ -75,24 +88,38 @@ protected StatefulSet desired(HiveCluster hiveCluster, spec.zookeeper().quorum(), null)); envVars.add(new EnvVar("HIVE_LLAP_DAEMON_SERVICE_HOSTS", llap.serviceHosts(), null)); + envVars.add(new EnvVar("LLAP_LOG4J2_PROPERTIES_FILE_NAME", + "llap-daemon-log4j2.properties", null)); // User-provided env vars (storage credentials, etc.) if (spec.envVars() != null) { 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() - ); - - Probe readinessProbe = buildTcpProbe(15004, llap.readinessProbe(), 15, 10, 3); + int managementPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT); + int shufflePort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT); + int webPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT); + int outputPort = ConfigUtils.getInt(llap.configOverrides(), + ConfigUtils.HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_KEY, null, + ConfigUtils.HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_DEFAULT); + + List ports = new ArrayList<>(); + ports.add(new ContainerPortBuilder() + .withName("management").withContainerPort(managementPort).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("shuffle").withContainerPort(shufflePort).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("web").withContainerPort(webPort).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("output").withContainerPort(outputPort).withProtocol("TCP").build()); + + Probe readinessProbe = buildTcpProbe(managementPort, llap.readinessProbe(), 15, 10, 3); String headlessServiceName = hiveCluster.getMetadata().getName() + "-llap"; @@ -106,8 +133,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 +142,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, autoscaling.metricsPort(), + 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 +167,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(llap.replicas()) + .withReplicas(replicas) .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) @@ -135,13 +175,13 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "llap") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("llap") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) @@ -159,20 +199,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..ff19afd5c023 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/MetastoreDeploymentDependent.java @@ -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; @@ -50,12 +51,23 @@ public class MetastoreDeploymentDependent extends HiveDependentResource { - public static final String COMPONENT = "metastore"; + public static final String COMPONENT = ConfigUtils.COMPONENT_METASTORE; 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) { @@ -65,7 +77,7 @@ protected Deployment desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "metastore", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.addAll(buildDbEnvVars(db)); if (spec.envVars() != null) { @@ -77,12 +89,15 @@ 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() - ); + int restPort = ConfigUtils.getInt( + spec.metastore().configOverrides(), + ConfigUtils.METASTORE_REST_HTTP_PORT_KEY, + null, ConfigUtils.METASTORE_REST_HTTP_PORT_DEFAULT); + List ports = new ArrayList<>(); + ports.add(new ContainerPortBuilder() + .withName("thrift").withContainerPort(thriftPort).withProtocol("TCP").build()); + ports.add(new ContainerPortBuilder() + .withName("rest").withContainerPort(restPort).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 +122,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, autoscaling.metricsPort(), + 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 +137,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,20 +150,20 @@ protected Deployment desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(spec.metastore().replicas()) + .withReplicas(replicas) .withNewSelector() .withMatchLabels(selectorLabels) .endSelector() .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "metastore") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("metastore") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) @@ -155,20 +183,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..fb4b588401c9 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/SchemaInitJobDependent.java @@ -34,6 +34,7 @@ import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.spec.DatabaseConfig; import org.apache.hive.kubernetes.operator.model.spec.SecretKeyRef; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.Labels; /** @@ -53,6 +54,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) { @@ -60,7 +67,7 @@ protected Job desired(HiveCluster hiveCluster, DatabaseConfig db = spec.metastore().database(); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "metastore", null)); + envVars.add(new EnvVar("SERVICE_NAME", ConfigUtils.COMPONENT_METASTORE, null)); envVars.add(new EnvVar("IS_RESUME", "false", null)); envVars.add(new EnvVar("HIVE_CUSTOM_CONF_DIR", CONF_MOUNT_PATH, null)); diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/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..fd81af716e6a 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/dependent/TezAmStatefulSetDependent.java @@ -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,7 +32,9 @@ import io.javaoperatorsdk.operator.processing.dependent.kubernetes.KubernetesDependent; import org.apache.hive.kubernetes.operator.model.HiveCluster; import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; +import org.apache.hive.kubernetes.operator.model.spec.AutoscalingSpec; import org.apache.hive.kubernetes.operator.model.spec.TezAmSpec; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.apache.hive.kubernetes.operator.util.HadoopXmlBuilder; import org.apache.hive.kubernetes.operator.util.HiveConfigBuilder; import org.apache.hive.kubernetes.operator.util.Labels; @@ -50,13 +53,24 @@ public class TezAmStatefulSetDependent extends HiveDependentResource { - public static final String COMPONENT = "tezam"; + public static final String COMPONENT = ConfigUtils.COMPONENT_TEZAM; private static final String SCRATCH_MOUNT_PATH = "/opt/hive/scratch"; public TezAmStatefulSetDependent() { 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) { @@ -66,7 +80,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, Labels.selectorForComponent(hiveCluster, COMPONENT); List envVars = new ArrayList<>(); - envVars.add(new EnvVar("SERVICE_NAME", "tezam", null)); + envVars.add(new EnvVar("SERVICE_NAME", COMPONENT, null)); envVars.add(new EnvVar("IS_RESUME", "true", null)); envVars.add(new EnvVar("HIVE_ZOOKEEPER_QUORUM", spec.zookeeper().quorum(), null)); @@ -98,8 +112,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 +121,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 +135,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 +148,7 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) .endMetadata() .withNewSpec() - .withReplicas(tezAm.replicas()) + .withReplicas(replicas) .withServiceName(headlessServiceName) .withNewSelector() .withMatchLabels(selectorLabels) @@ -135,16 +156,17 @@ protected StatefulSet desired(HiveCluster hiveCluster, .withNewTemplate() .withNewMetadata() .withLabels(Labels.forComponent(hiveCluster, COMPONENT)) - .addToAnnotations("kubectl.kubernetes.io/default-container", "tezam") + .addToAnnotations("kubectl.kubernetes.io/default-container", COMPONENT) .addToAnnotations("hive.apache.org/config-hash", configHash) .endMetadata() .withNewSpec() .withInitContainers(initContainers) .addNewContainer() - .withName("tezam") + .withName(COMPONENT) .withImage(spec.image()) .withImagePullPolicy(spec.imagePullPolicy()) .withEnv(envVars) + .withPorts(ports) .withResources(buildResources(tezAm.resources())) .withVolumeMounts(volumeMounts) .endContainer() @@ -157,20 +179,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..a6d9c709880c 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterSpec.java @@ -28,6 +28,7 @@ import io.fabric8.kubernetes.api.model.EnvVar; import io.fabric8.kubernetes.api.model.Volume; import io.fabric8.kubernetes.api.model.VolumeMount; +import org.apache.hive.kubernetes.operator.model.spec.AutoSuspendSpec; import org.apache.hive.kubernetes.operator.model.spec.HadoopSpec; import org.apache.hive.kubernetes.operator.model.spec.HiveServer2Spec; import org.apache.hive.kubernetes.operator.model.spec.LlapSpec; @@ -73,14 +74,31 @@ public record HiveClusterSpec( "Volume mounts added to all component containers " + "(e.g., mounting a GCS key file at /etc/gcs/key.json)") @SchemaFrom(type = Object[].class) @PreserveUnknownFields - List volumeMounts) { + List volumeMounts, + @JsonPropertyDescription("Auto-suspend configuration. When enabled and all components " + + "are idle for the configured timeout, the cluster scales to 0 replicas.") + AutoSuspendSpec autoSuspend, + @JsonPropertyDescription("When true, the cluster is immediately suspended (all components " + + "scaled to 0). Set to false to wake a suspended cluster.") + Boolean suspend) { public HiveClusterSpec { Objects.requireNonNull(zookeeper, "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(); volumeMounts = volumeMounts != null ? volumeMounts : List.of(); + autoSuspend = autoSuspend != null ? autoSuspend : new AutoSuspendSpec(false, 15, true); + suspend = suspend != null ? suspend : false; } + } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java index d2432dda2246..ea6758309f11 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/HiveClusterStatus.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import io.fabric8.crd.generator.annotation.PrinterColumn; import io.fabric8.kubernetes.api.model.Condition; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; @@ -33,6 +34,13 @@ public class HiveClusterStatus { private ComponentStatus llap; private ComponentStatus tezAm; private Long observedGeneration; + @PrinterColumn(name = "Phase") + private String clusterPhase; + private String idleSince; + @PrinterColumn(name = "Idle (min)") + private Integer idleForMinutes; + @PrinterColumn(name = "Suspended Since", priority = 1) + private String suspendedSince; public List getConditions() { return conditions; @@ -82,6 +90,38 @@ public void setObservedGeneration(Long observedGeneration) { this.observedGeneration = observedGeneration; } + public String getClusterPhase() { + return clusterPhase; + } + + public void setClusterPhase(String clusterPhase) { + this.clusterPhase = clusterPhase; + } + + public String getIdleSince() { + return idleSince; + } + + public void setIdleSince(String idleSince) { + this.idleSince = idleSince; + } + + public Integer getIdleForMinutes() { + return idleForMinutes; + } + + public void setIdleForMinutes(Integer idleForMinutes) { + this.idleForMinutes = idleForMinutes; + } + + public String getSuspendedSince() { + return suspendedSince; + } + + public void setSuspendedSince(String suspendedSince) { + this.suspendedSince = suspendedSince; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -96,11 +136,16 @@ public boolean equals(Object o) { java.util.Objects.equals(metastore, that.metastore) && java.util.Objects.equals(hiveServer2, that.hiveServer2) && java.util.Objects.equals(llap, that.llap) && - java.util.Objects.equals(tezAm, that.tezAm); + java.util.Objects.equals(tezAm, that.tezAm) && + java.util.Objects.equals(clusterPhase, that.clusterPhase) && + java.util.Objects.equals(idleSince, that.idleSince) && + java.util.Objects.equals(idleForMinutes, that.idleForMinutes) && + java.util.Objects.equals(suspendedSince, that.suspendedSince); } @Override public int hashCode() { - return java.util.Objects.hash(conditions, metastore, hiveServer2, llap, tezAm, observedGeneration); + return java.util.Objects.hash(conditions, metastore, hiveServer2, llap, tezAm, + observedGeneration, clusterPhase, idleSince, idleForMinutes, suspendedSince); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java new file mode 100644 index 000000000000..196d57e8bc86 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoSuspendSpec.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.kubernetes.operator.model.spec; + +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import io.fabric8.generator.annotation.Default; + +/** + * Auto-suspend configuration. When enabled and all components are idle for the + * configured timeout, the operator scales the entire cluster to 0 replicas. + * Requires autoscaling to be enabled on all active components. + */ +public record AutoSuspendSpec( + @JsonPropertyDescription("Whether auto-suspend is enabled. Requires autoscaling " + + "to be enabled on all active components (HS2, LLAP if enabled, TezAM if enabled, " + + "and HMS if includeMetastore is true).") + @Default("false") + Boolean enabled, + @JsonPropertyDescription("Minutes of idle time (HS2=0 sessions, LLAP/TezAM at minReplicas) " + + "before the cluster auto-suspends.") + @Default("15") + Integer idleTimeoutMinutes, + @JsonPropertyDescription("Whether Metastore participates in auto-suspend. " + + "When false, HMS stays at minReplicas during suspend and HMS autoscaling " + + "is not required for auto-suspend to activate.") + @Default("true") + Boolean includeMetastore) { + + public AutoSuspendSpec { + enabled = enabled != null ? enabled : false; + idleTimeoutMinutes = idleTimeoutMinutes != null ? idleTimeoutMinutes : 15; + includeMetastore = includeMetastore != null ? includeMetastore : true; + } + + public boolean isEnabled() { + return enabled; + } +} diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/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..fac4b016766a --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/AutoscalingSpec.java @@ -0,0 +1,87 @@ +/* + * 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; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; + +/** Autoscaling configuration for a Hive component. The operator scrapes JMX metrics directly from pods. */ +public record AutoscalingSpec( + @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, + @JsonPropertyDescription("CPU percentage (0-100) that triggers scale-up. " + + "Only applies to HS2 and HMS. Set to 0 to disable CPU-based scaling.") + @Default("90") + Integer cpuScaleUpThreshold, + @JsonPropertyDescription("CPU percentage (0-100) below which scale-down is considered. " + + "Only applies to HS2 and HMS.") + @Default("30") + Integer cpuScaleDownThreshold, + @JsonPropertyDescription("Port on which the Prometheus JMX Exporter serves metrics. " + + "The operator scrapes this port on each pod for autoscaling decisions.") + @Default("9404") + Integer metricsPort) { + + public AutoscalingSpec { + enabled = enabled != null ? enabled : false; + 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; + cpuScaleUpThreshold = cpuScaleUpThreshold != null ? cpuScaleUpThreshold : 90; + cpuScaleDownThreshold = cpuScaleDownThreshold != null ? cpuScaleDownThreshold : 30; + metricsPort = metricsPort != null ? metricsPort : ConfigUtils.PROMETHEUS_JMX_EXPORTER_PORT; + } + + 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..5f00a5c1dde5 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/HiveServer2Spec.java @@ -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, 90, 30, null); } } diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java index 17ff5967ff9a..b4865d66ce43 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/LlapSpec.java @@ -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, 0, 0, null); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java index 307c17221ee7..e1a0ac1452ae 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/MetastoreSpec.java @@ -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, 90, 30, null); } public boolean isEnabled() { diff --git a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java index a0494c2c5e73..606b0de14053 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/spec/TezAmSpec.java @@ -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, 0, 0, null); } 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..5a074b7b222e --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/model/status/AutoscalingStatus.java @@ -0,0 +1,120 @@ +/* + * 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 Double currentCpuPercent; + private Integer cpuScaleUpThreshold; + private Integer cpuProposedReplicas; + 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 Double getCurrentCpuPercent() { + return currentCpuPercent; + } + + public void setCurrentCpuPercent(Double currentCpuPercent) { + this.currentCpuPercent = currentCpuPercent; + } + + public Integer getCpuScaleUpThreshold() { + return cpuScaleUpThreshold; + } + + public void setCpuScaleUpThreshold(Integer cpuScaleUpThreshold) { + this.cpuScaleUpThreshold = cpuScaleUpThreshold; + } + + public Integer getCpuProposedReplicas() { + return cpuProposedReplicas; + } + + public void setCpuProposedReplicas(Integer cpuProposedReplicas) { + this.cpuProposedReplicas = cpuProposedReplicas; + } + + public int getProposedReplicas() { + return proposedReplicas; + } + + 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) + && Objects.equals(currentCpuPercent, that.currentCpuPercent) + && Objects.equals(cpuScaleUpThreshold, that.cpuScaleUpThreshold) + && Objects.equals(cpuProposedReplicas, that.cpuProposedReplicas) + && proposedReplicas == that.proposedReplicas + && Objects.equals(lastScaleTime, that.lastScaleTime); + } + + @Override + public int hashCode() { + return Objects.hash(currentMetricValue, scaleUpThreshold, + currentCpuPercent, cpuScaleUpThreshold, cpuProposedReplicas, + 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..4d7a097c8a1d 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveClusterReconciler.java @@ -18,48 +18,39 @@ 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.Cleaner; import io.javaoperatorsdk.operator.api.reconciler.ControllerConfiguration; +import io.javaoperatorsdk.operator.api.reconciler.DeleteControl; import io.javaoperatorsdk.operator.api.reconciler.ErrorStatusUpdateControl; import io.javaoperatorsdk.operator.api.reconciler.Reconciler; import io.javaoperatorsdk.operator.api.reconciler.UpdateControl; -import 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.BackgroundMetricsScraper; +import org.apache.hive.kubernetes.operator.autoscaling.HiveClusterAutoscaler; +import org.apache.hive.kubernetes.operator.autoscaling.MetricsCache; +import org.apache.hive.kubernetes.operator.autoscaling.MetricsScraper; +import org.apache.hive.kubernetes.operator.autoscaling.PodMetrics; import org.apache.hive.kubernetes.operator.model.HiveCluster; +import org.apache.hive.kubernetes.operator.model.HiveClusterSpec; import org.apache.hive.kubernetes.operator.model.HiveClusterStatus; +import org.apache.hive.kubernetes.operator.model.spec.AutoSuspendSpec; +import org.apache.hive.kubernetes.operator.model.status.AutoscalingStatus; import org.apache.hive.kubernetes.operator.model.status.ComponentStatus; +import org.apache.hive.kubernetes.operator.util.ConfigUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,66 +59,143 @@ * 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 { +public class HiveClusterReconciler + implements Reconciler, Cleaner { private static final Logger LOG = LoggerFactory.getLogger(HiveClusterReconciler.class); + private volatile HiveClusterAutoscaler autoscaler; + private volatile BackgroundMetricsScraper bgScraper; + @Override public UpdateControl reconcile(HiveCluster resource, Context context) { - 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)) { - return UpdateControl.noUpdate(); + // --- Suspend / Wake evaluation (works regardless of autoscaling) --- + KubernetesClient client = context.getClient(); + SuspendAction action = evaluateSuspendState(resource, existingStatus, client); + int rescheduleSeconds = 0; + + switch (action) { + case SUSPEND_NOW: + suspendCluster(resource); + boolean manual = resource.getSpec().suspend(); + // Auto-suspend: set spec.suspend=true so the cluster stays suspended + // until the user explicitly sets it to false. + // The spec patch triggers a watch event → immediate re-reconcile where + // STAY_SUSPENDED sets the status cleanly. + if (!manual) { + patchSuspendSpec(client, resource, true); + return UpdateControl.noUpdate(); + } + String reason = "ManualSuspend"; + newStatus.setClusterPhase("Suspended"); + newStatus.setSuspendedSince(Instant.now().toString()); + newStatus.setIdleSince(null); + newStatus.getConditions().add(buildCondition("Suspended", "True", reason, + "Cluster suspended via spec.suspend", + existingStatus != null ? existingStatus.getConditions() : Collections.emptyList())); + rescheduleSeconds = 30; + break; + + case STAY_SUSPENDED: + newStatus.setClusterPhase("Suspended"); + newStatus.setSuspendedSince(existingStatus != null ? existingStatus.getSuspendedSince() : null); + newStatus.setIdleSince(null); + newStatus.getConditions().add(buildCondition("Suspended", "True", "Suspended", + "Cluster is suspended", + existingStatus != null ? existingStatus.getConditions() : Collections.emptyList())); + rescheduleSeconds = 30; + break; + + case WAKE: + wakeCluster(resource); + newStatus.setClusterPhase("Running"); + newStatus.setSuspendedSince(null); + newStatus.setIdleSince(null); + newStatus.getConditions().add(buildCondition("Suspended", "False", "Woken", + "Cluster woken up", + existingStatus != null ? existingStatus.getConditions() : Collections.emptyList())); + rescheduleSeconds = anyAutoscalingEnabled(resource.getSpec()) + ? getMinScrapeInterval(resource.getSpec()) : 30; + break; + + case IDLE_START: + newStatus.setClusterPhase("Idle"); + newStatus.setIdleSince(Instant.now().toString()); + newStatus.setIdleForMinutes(0); + newStatus.setSuspendedSince(null); + break; + + case IDLE_WAITING: + String idleSince = existingStatus != null ? existingStatus.getIdleSince() : null; + newStatus.setClusterPhase("Idle"); + newStatus.setIdleSince(idleSince); + newStatus.setIdleForMinutes(idleSince != null + ? (int) Duration.between(Instant.parse(idleSince), Instant.now()).toMinutes() : 0); + newStatus.setSuspendedSince(null); + break; + + case RUNNING: + default: + newStatus.setClusterPhase("Running"); + newStatus.setIdleSince(null); + newStatus.setIdleForMinutes(null); + newStatus.setSuspendedSince(null); + break; + } + + // --- Autoscaling evaluation (only when enabled and not suspended) --- + if (rescheduleSeconds == 0 && anyAutoscalingEnabled(resource.getSpec())) { + 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()); + } + applyAutoscalingStatuses(newStatus, eval.statuses()); + // Reschedule sooner if a two-phase scale-down is pending annotation propagation + rescheduleSeconds = scaler.hasPendingScaleDowns() + ? 2 : getMinScrapeInterval(resource.getSpec()); } + // --- Single exit point for status update --- + boolean statusNowChanged = !statusEqualsIgnoringTimestamps(existingStatus, newStatus); + if (!statusNowChanged && rescheduleSeconds == 0) { + return UpdateControl.noUpdate(); + } resource.setStatus(newStatus); + if (rescheduleSeconds > 0) { + return UpdateControl.patchStatus(resource) + .rescheduleAfter(Duration.ofSeconds(rescheduleSeconds)); + } return UpdateControl.patchStatus(resource); } + @Override + public DeleteControl cleanup(HiveCluster resource, Context context) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + if (autoscaler != null) { + autoscaler.cleanupCluster(ns, name); + } + if (bgScraper != null) { + bgScraper.unregisterCluster(ns, name); + } + LOG.info("Cleaned up autoscaler state for deleted cluster {}/{}", ns, name); + return DeleteControl.defaultDelete(); + } + @Override public ErrorStatusUpdateControl updateErrorStatus(HiveCluster resource, Context context, Exception e) { - 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 +240,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 +260,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 +276,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 +311,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 +368,377 @@ 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(ConfigUtils.COMPONENT_HIVESERVER2) && status.getHiveServer2() != null) { + status.getHiveServer2().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_HIVESERVER2)); + } + if (statuses.containsKey(ConfigUtils.COMPONENT_METASTORE) && status.getMetastore() != null) { + status.getMetastore().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_METASTORE)); + } + if (statuses.containsKey(ConfigUtils.COMPONENT_LLAP) && status.getLlap() != null) { + status.getLlap().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_LLAP)); + } + if (statuses.containsKey(ConfigUtils.COMPONENT_TEZAM) && status.getTezAm() != null) { + status.getTezAm().setAutoscaling(statuses.get(ConfigUtils.COMPONENT_TEZAM)); + } + } + + // --- Autoscaling helpers --- + + private HiveClusterAutoscaler getOrCreateAutoscaler(KubernetesClient client) { + if (autoscaler == null) { + MetricsScraper scraper = new MetricsScraper(client); + MetricsCache metricsCache = new MetricsCache(); + bgScraper = new BackgroundMetricsScraper(scraper, metricsCache); + autoscaler = new HiveClusterAutoscaler(scraper, bgScraper, metricsCache); + } + 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; + try { + if (ConfigUtils.COMPONENT_LLAP.equals(component) || ConfigUtils.COMPONENT_TEZAM.equals(component)) { + client.apps().statefulSets().inNamespace(namespace).withName(workloadName).scale(replicas); + } else { + client.apps().deployments().inNamespace(namespace).withName(workloadName).scale(replicas); + } + LOG.info("Scaled {}/{} to {} replicas", namespace, workloadName, replicas); + } catch (Exception e) { + LOG.debug("Could not scale {}/{}: {}", namespace, workloadName, e.getMessage()); + } + } + + private void patchSuspendSpec(KubernetesClient client, HiveCluster resource, boolean suspend) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + client.resources(HiveCluster.class).inNamespace(ns).withName(name) + .edit(hc -> { + // Records are immutable so we build a new spec with the updated suspend value + HiveClusterSpec oldSpec = hc.getSpec(); + HiveClusterSpec newSpec = new HiveClusterSpec( + oldSpec.image(), oldSpec.imagePullPolicy(), oldSpec.metastore(), + oldSpec.hiveServer2(), oldSpec.llap(), oldSpec.tezAm(), oldSpec.zookeeper(), + oldSpec.hadoop(), oldSpec.envVars(), oldSpec.externalJars(), + oldSpec.volumes(), oldSpec.volumeMounts(), oldSpec.autoSuspend(), suspend); + hc.setSpec(newSpec); + return hc; + }); + LOG.info("Patched spec.suspend={} on {}/{}", suspend, ns, name); + } + + // --- Auto-Suspend / Wake --- + + enum SuspendAction { RUNNING, IDLE_START, IDLE_WAITING, SUSPEND_NOW, STAY_SUSPENDED, WAKE } + + private SuspendAction evaluateSuspendState(HiveCluster resource, + HiveClusterStatus existingStatus, KubernetesClient client) { + + // 1. Manual suspend: spec.suspend = true → suspend immediately + if (resource.getSpec().suspend()) { + if (existingStatus != null && "Suspended".equals(existingStatus.getClusterPhase())) { + return SuspendAction.STAY_SUSPENDED; + } + return SuspendAction.SUSPEND_NOW; + } + + // 2. Currently suspended and spec.suspend = false → wake + if (existingStatus != null && "Suspended".equals(existingStatus.getClusterPhase())) { + return SuspendAction.WAKE; + } + + // 3. Auto-suspend evaluation (only if enabled and all autoscaling is on) + AutoSuspendSpec autoSuspend = resource.getSpec().autoSuspend(); + if (!autoSuspend.isEnabled()) { + LOG.debug("Auto-suspend disabled"); + return SuspendAction.RUNNING; + } + if (!allAutoscalingEnabled(resource.getSpec())) { + LOG.debug("Auto-suspend skipped: not all components have autoscaling enabled"); + return SuspendAction.RUNNING; + } + + // 4. Check idle conditions + boolean allIdle = isClusterIdle(resource, client); + if (!allIdle) { + return SuspendAction.RUNNING; + } + + // 5. Check idle duration + String idleSince = existingStatus != null ? existingStatus.getIdleSince() : null; + if (idleSince == null) { + return SuspendAction.IDLE_START; + } + + Instant idleStart = Instant.parse(idleSince); + if (Duration.between(idleStart, Instant.now()).toMinutes() >= autoSuspend.idleTimeoutMinutes()) { + return SuspendAction.SUSPEND_NOW; + } + + return SuspendAction.IDLE_WAITING; + } + + + private boolean isClusterIdle(HiveCluster resource, KubernetesClient client) { + HiveClusterSpec spec = resource.getSpec(); + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + + // All components must be at minReplicas + if (spec.llap().isEnabled() + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_LLAP, true, + spec.llap().autoscaling().minReplicas())) { + return false; + } + if (spec.tezAm().isEnabled() + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_TEZAM, true, + spec.tezAm().autoscaling().minReplicas())) { + return false; + } + if (!isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_HIVESERVER2, false, + Math.max(1, spec.hiveServer2().autoscaling().minReplicas()))) { + return false; + } + + // HS2 must have 0 open sessions. + // If metrics scrape fails (empty list), assume NOT idle to prevent accidental suspend. + HiveClusterAutoscaler scaler = getOrCreateAutoscaler(client); + List hs2Metrics = scaler.getHs2MetricsFromCache(resource); + if (hs2Metrics.isEmpty()) { + LOG.debug("Idle check: HS2 metrics unavailable, assuming not idle"); + return false; + } + int totalSessions = hs2Metrics.stream() + .mapToInt(pm -> pm.metrics().getOrDefault("hs2_open_sessions", 0.0).intValue()) + .sum(); + if (totalSessions > 0) { + LOG.debug("Idle check failed: HS2 has {} open sessions", totalSessions); + return false; + } + + // HMS must be at minReplicas (only checked if includeMetastore=true) + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore() + && !isAtMinReplicas(client, ns, name + "-" + ConfigUtils.COMPONENT_METASTORE, false, + Math.max(1, spec.metastore().autoscaling().minReplicas()))) { + return false; + } + + return true; + } + + /** Returns true if the workload is absent or its replicas <= minReplicas. */ + private boolean isAtMinReplicas(KubernetesClient client, String ns, + String workloadName, boolean statefulSet, int minReplicas) { + try { + Integer currentReplicas = null; + if (statefulSet) { + var ss = client.apps().statefulSets().inNamespace(ns).withName(workloadName).get(); + if (ss != null && ss.getSpec() != null) { + currentReplicas = ss.getSpec().getReplicas(); + } + } else { + var deploy = client.apps().deployments().inNamespace(ns).withName(workloadName).get(); + if (deploy != null && deploy.getSpec() != null) { + currentReplicas = deploy.getSpec().getReplicas(); + } + } + if (currentReplicas != null && currentReplicas > minReplicas) { + LOG.debug("Idle check failed: {} replicas {} > min {}", workloadName, currentReplicas, minReplicas); + return false; + } + return true; + } catch (Exception e) { + LOG.debug("Idle check: could not read {}: {}", workloadName, e.getMessage()); + return true; + } + } + + private void suspendCluster(HiveCluster resource) { + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + HiveClusterSpec spec = resource.getSpec(); + + // Set MANAGED_REPLICAS to 0 so autoscaler doesn't fight the suspend. + // Actual scaling to 0 is handled by the DependentResources which check + // spec.suspend() in resolveReplicaCount(). + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_HIVESERVER2, 0); + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_METASTORE, 0); + } + if (spec.llap().isEnabled()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_LLAP, 0); + } + if (spec.tezAm().isEnabled()) { + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_TEZAM, 0); + } + + LOG.info("Cluster {}/{} suspended", ns, name); + } + + private void wakeCluster(HiveCluster resource) { + HiveClusterSpec spec = resource.getSpec(); + String ns = resource.getMetadata().getNamespace(); + String name = resource.getMetadata().getName(); + + // Set MANAGED_REPLICAS to wake values. The JOSDK workflow will recreate + // the dependent resources (Deployments/StatefulSets) on the next reconcile + // and use these values for spec.replicas. We don't call patchReplicas() + // because the workloads may have been garbage-collected while suspended. + int hs2Min = Math.max(1, spec.hiveServer2().autoscaling().minReplicas()); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_HIVESERVER2, hs2Min); + + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore()) { + int hmsMin = Math.max(1, spec.metastore().autoscaling().minReplicas()); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_METASTORE, hmsMin); + } + + if (spec.llap().isEnabled()) { + int llapWake = spec.llap().autoscaling().minReplicas(); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_LLAP, llapWake); + } + + if (spec.tezAm().isEnabled()) { + int tezWake = spec.tezAm().autoscaling().minReplicas(); + HiveClusterAutoscaler.setManagedReplicas(ns, name, ConfigUtils.COMPONENT_TEZAM, tezWake); + } + + LOG.info("Cluster {}/{} woken up — restored to minReplicas", ns, name); + } + + private static boolean allAutoscalingEnabled(HiveClusterSpec spec) { + if (!spec.hiveServer2().autoscaling().isEnabled()) { + return false; + } + // Skip HMS check if includeMetastore=false (HMS doesn't participate in suspend) + if (spec.metastore().isEnabled() && spec.autoSuspend().includeMetastore() + && !spec.metastore().autoscaling().isEnabled()) { + return false; + } + if (spec.llap().isEnabled() && !spec.llap().autoscaling().isEnabled()) { + return false; + } + if (spec.tezAm().isEnabled() && !spec.tezAm().autoscaling().isEnabled()) { + return false; + } + return true; + } } 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..c8ee85759678 --- /dev/null +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/reconciler/HiveWorkflowSpec.java @@ -0,0 +1,280 @@ +/* + * 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 { + + // Dependent resource spec names (used as identifiers and dependency references) + private static final String HADOOP_CONFIGMAP = "hadoop-configmap"; + private static final String METASTORE_CONFIGMAP = "metastore-configmap"; + private static final String HIVESERVER2_CONFIGMAP = "hiveserver2-configmap"; + private static final String LLAP_CONFIGMAP = "llap-configmap"; + private static final String SCHEMA_INIT_JOB = "schema-init-job"; + private static final String METASTORE_DEPLOYMENT = "metastore-deployment"; + private static final String METASTORE_SERVICE = "metastore-service"; + private static final String HIVESERVER2_DEPLOYMENT = "hiveserver2-deployment"; + private static final String HIVESERVER2_SERVICE = "hiveserver2-service"; + private static final String LLAP_STATEFULSET = "llap-statefulset"; + private static final String LLAP_SERVICE = "llap-service"; + private static final String TEZAM_SERVICE = "tezam-service"; + private static final String TEZAM_STATEFULSET = "tezam-statefulset"; + private static final String SCRATCH_PVC = "scratch-pvc"; + private static final String HS2_PDB = "hs2-pdb"; + private static final String METASTORE_PDB = "metastore-pdb"; + private static final String LLAP_PDB = "llap-pdb"; + private static final String TEZAM_PDB = "tezam-pdb"; + + private static final Condition METASTORE_ENABLED = + (dr, primary, ctx) -> primary.getSpec().metastore().isEnabled(); + + 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..c3d824763c60 100644 --- a/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java +++ b/packaging/src/kubernetes/src/java/org/apache/hive/kubernetes/operator/util/ConfigUtils.java @@ -25,6 +25,12 @@ public final class ConfigUtils { private ConfigUtils() { } + // Component names used across the operator for labels, resource naming, and autoscaling keys. + public static final String COMPONENT_HIVESERVER2 = "hiveserver2"; + public static final String COMPONENT_METASTORE = "metastore"; + public static final String COMPONENT_LLAP = "llap"; + public static final String COMPONENT_TEZAM = "tezam"; + public static final String METASTORE_THRIFT_PORT_KEY = "metastore.thrift.port"; public static final String METASTORE_THRIFT_PORT_HIVE_KEY = "hive.metastore.port"; public static final int METASTORE_THRIFT_PORT_DEFAULT = 9083; @@ -73,14 +79,65 @@ private ConfigUtils() { public static final String HIVE_LLAP_DAEMON_NUM_EXECUTORS_KEY = "hive.llap.daemon.num.executors"; + public static final String HIVE_LLAP_DAEMON_RPC_PORT_KEY = "hive.llap.daemon.rpc.port"; + public static final int HIVE_LLAP_DAEMON_RPC_PORT_DEFAULT = 15001; + + public static final String HIVE_LLAP_MANAGEMENT_RPC_PORT_KEY = "hive.llap.management.rpc.port"; + public static final int HIVE_LLAP_MANAGEMENT_RPC_PORT_DEFAULT = 15004; + + public static final String HIVE_LLAP_DAEMON_SHUFFLE_PORT_KEY = "hive.llap.daemon.yarn.shuffle.port"; + public static final int HIVE_LLAP_DAEMON_SHUFFLE_PORT_DEFAULT = 15551; + + public static final String HIVE_LLAP_DAEMON_WEB_PORT_KEY = "hive.llap.daemon.web.port"; + public static final int HIVE_LLAP_DAEMON_WEB_PORT_DEFAULT = 15002; + + public static final String HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_KEY = "hive.llap.daemon.output.service.port"; + public static final int HIVE_LLAP_DAEMON_OUTPUT_SERVICE_PORT_DEFAULT = 15003; + + public static final String METASTORE_SERVER_TRANSPORT_MODE_KEY = "metastore.server.thrift.transport.mode"; + public static final String METASTORE_SERVER_TRANSPORT_MODE_DEFAULT = "http"; + + 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 METASTORE_REST_HTTP_PORT_KEY = "metastore.rest.http.port"; + public static final int METASTORE_REST_HTTP_PORT_DEFAULT = 9001; + public static final String HIVE_METASTORE_URIS_KEY = "hive.metastore.uris"; + public static final String HIVE_SERVER2_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 +172,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() {