From 86a4989957bf718ac10d65980588a5d00af26733 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:01:08 -0600 Subject: [PATCH 01/21] docs: add design spec for batch stash shuffle optimization Design for avoiding unnecessary Arrow FFI import/export when passing batches between two native plans (issue #3925). Uses a native-side batch registry to pass opaque handles through the JVM instead of full Arrow FFI round-trips. --- ...batch-stash-shuffle-optimization-design.md | 254 ++++++++++++++++++ 1 file changed, 254 insertions(+) create mode 100644 docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md diff --git a/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md b/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md new file mode 100644 index 0000000000..7523ac841c --- /dev/null +++ b/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md @@ -0,0 +1,254 @@ +# Batch Stash: Avoid FFI Import/Export Between Native Plans + +**Issue**: https://github.com/apache/datafusion-comet/issues/3925 +**Date**: 2026-04-12 + +## Problem + +When Comet has a native ShuffleWriter and a native child plan, batches are created in native code, +exported to JVM via Arrow FFI, then immediately imported back to native for the shuffle writer. The +JVM never reads the data. This round-trip is unnecessary overhead. + +### Current flow (per batch) + +``` +Native child plan produces RecordBatch + -> prepare_output(): FFI export each column via move_to_spark() [native -> JVM] + -> NativeUtil.getNextBatch(): import into CometVector/ColumnarBatch [JVM] + -> CometBatchIterator.next(): exportBatch() via Data.exportVector() [JVM -> native] + -> ScanExec.get_next(): import via ArrayData::from_spark() + copy [native] +Shuffle writer processes RecordBatch +``` + +Each batch crosses the JNI/FFI boundary 4 times and gets copied at least twice. + +## Solution + +Pass batches as opaque handles through the JVM instead of doing Arrow FFI export/import. + +### New flow (per batch) + +``` +Native child plan produces RecordBatch + -> stash RecordBatch in global registry, return handle (u64) [native] + -> JVM receives handle as jlong [JVM] + -> JVM passes handle to shuffle writer's ScanExec [JVM -> native] + -> ScanExec retrieves RecordBatch from registry [native] +Shuffle writer processes RecordBatch +``` + +Two lightweight JNI calls passing a single long value. No Arrow FFI, no data copying. + +## Design + +### Approach + +Keep the current two-separate-native-plans architecture (one for the child plan, one for the +shuffle writer). Add a "batch stash" registry on the native side. When the child plan produces a +batch, stash it and return a handle. The shuffle writer's ScanExec retrieves the batch using the +handle. + +Detection is automatic. No new config flags. + +### Component 1: BatchStash (Rust) + +New file: `native/core/src/execution/batch_stash.rs` + +A global thread-safe registry mapping u64 handles to RecordBatch values. + +- Uses `Mutex>` (simple and sufficient since contention is minimal: + each task thread produces and consumes one batch at a time) +- `AtomicU64` counter for generating unique handles +- `stash(batch: RecordBatch) -> u64`: inserts and returns handle +- `take(handle: u64) -> Option`: removes and returns (consumed exactly once) +- `clear_for_context(id: i64)`: cleanup method for error/abort paths (not strictly needed since + batches are consumed one-at-a-time, but provides safety) + +Memory: at most one batch is stashed at a time per task (produce one, consume one). Memory +footprint matches the current approach. + +### Component 2: executePlanBatchHandle (JNI) + +Add a new JNI function to `Native.java` / `jni_api.rs`: + +```java +// Native.java +native long executePlanBatchHandle(int stageId, int partition, long plan); +``` + +The Rust implementation is nearly identical to `Java_org_apache_comet_Native_executePlan` but +replaces the `prepare_output()` call (which does FFI export) with `batch_stash::stash(batch)`. +Returns the handle (positive u64 cast to jlong) or -1 for EOF. + +Does not take `array_addrs` or `schema_addrs` parameters since no FFI export occurs. + +### Component 3: CometExecIterator Stash Mode + +Add a `stashMode` flag to `CometExecIterator`. When enabled: + +- `getNextBatch` calls `nativeLib.executePlanBatchHandle()` instead of `executePlan()` +- Stores the handle internally instead of importing Arrow arrays into a ColumnarBatch +- Exposes `def nextHandle(): Long` that returns the handle or -1 for EOF + +The existing `Iterator[ColumnarBatch]` interface stays unchanged for the non-stash path. The stash +path is used only by the shuffle writer, which calls `nextHandle()` directly. + +`enableStashMode()` is called by `CometNativeShuffleWriter` after detecting that its input comes +from a CometExecIterator. + +### Component 4: CometHandleBatchIterator (Java) + +New file: `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` + +Replaces `CometBatchIterator` for the stash path. Called from native ScanExec via JNI: + +```java +public class CometHandleBatchIterator { + private final CometExecIterator source; + + public CometHandleBatchIterator(CometExecIterator source) { + this.source = source; + } + + // Called by native ScanExec via JNI + // Returns batch handle or -1 for EOF + public long nextHandle() { + return source.nextHandle(); + } +} +``` + +The native JNI bridge gets a corresponding struct with a `method_next_handle` JMethodID. + +### Component 5: ScanExec Handle Path + +Modify `ScanExec.get_next()` to check whether its input source is a `CometHandleBatchIterator`. +If so: + +1. Call `nextHandle()` via JNI to get a handle +2. Call `batch_stash::take(handle)` to retrieve the RecordBatch +3. Wrap it as an InputBatch directly (no FFI import, no copy, no dictionary unpacking) + +Detection: during `createPlan` in `PhysicalPlanner`, when the planner encounters a Scan whose +source is `"ShuffleWriterInput"` and whose input object is a `CometHandleBatchIterator`, it sets a +flag on the ScanExec indicating handle mode. + +### Component 6: Detection Mechanism + +The challenge: `CometNativeShuffleWriter.write()` receives `Iterator[Product2[K, V]]` from Spark's +shuffle framework. The original `CometExecIterator` is wrapped in `rdd.map((0, _))` which creates +a MappedIterator, losing the type. + +Solution: preserve the CometExecIterator reference through the RDD. + +New class `CometShuffleWriterInputIterator`: + +```scala +class CometShuffleWriterInputIterator( + underlying: Iterator[ColumnarBatch], + val nativeIterator: Option[CometExecIterator] +) extends Iterator[Product2[Int, ColumnarBatch]] { + def hasNext: Boolean = underlying.hasNext + def next(): Product2[Int, ColumnarBatch] = (0, underlying.next()) +} +``` + +In `CometShuffleExchangeExec.prepareShuffleDependency`, replace `rdd.map((0, _))` with: + +```scala +val wrappedRDD = rdd.mapPartitions { iter => + val nativeIter = iter match { + case cei: CometExecIterator => Some(cei) + case _ => None + } + new CometShuffleWriterInputIterator(iter, nativeIter) +} +``` + +Spark's `ShuffleWriteProcessor.write()` passes `rdd.iterator(partition, context)` directly to +`ShuffleWriter.write()`, so the type is preserved. + +In `CometNativeShuffleWriter.write()`: + +```scala +val nativeIter: Option[CometExecIterator] = inputs match { + case swi: CometShuffleWriterInputIterator => swi.nativeIterator + case _ => None +} + +if (nativeIter.isDefined) { + // Stash mode: enable stash on child iterator, use handle-based input + nativeIter.get.enableStashMode() + val handleIter = new CometHandleBatchIterator(nativeIter.get) + // Create shuffle writer's CometExecIterator with handleIter as input +} else { + // Fallback: existing FFI path (unchanged) +} +``` + +### Component 7: CometNativeShuffleWriter Changes + +When stash mode is detected, the shuffle writer creates its CometExecIterator differently: + +- Pass `CometHandleBatchIterator` as the input iterator (instead of `CometBatchIterator`) +- The shuffle writer's native plan (Scan -> ShuffleWriter) still has the same structure +- The difference is that the Scan's input source is a `CometHandleBatchIterator` instead of + `CometBatchIterator`, so ScanExec uses the handle path + +The `getCometIterator` method (or a new overload) needs to accept handle-based input iterators. +CometExecIterator already takes `Array[Object]` as `inputIterators`, so we can pass the +`CometHandleBatchIterator` directly as an Object, bypassing the normal CometBatchIterator wrapping. + +## Edge Cases + +1. **Non-native child plan**: `nativeIterator` is `None`. Falls back to the existing FFI path. + No behavior change. + +2. **Error during shuffle write**: If the shuffle writer aborts, any stashed batch that was never + consumed would leak. The `releasePlan` cleanup path should call `batch_stash::take()` for any + outstanding handle to prevent this. In practice, at most one batch is stashed at a time, and it + is consumed before the next is produced. + +3. **Empty batches / EOF**: `executePlanBatchHandle` returns -1 for EOF, same convention as + `executePlan`. Zero-row batches get stashed normally. + +4. **Multiple scans in shuffle writer plan**: The shuffle writer's native plan has exactly one + Scan child (`ShuffleWriterInput`), so there is always exactly one input iterator. + +5. **Memory pressure**: Batches are consumed one at a time (produce one, consume one), so at most + one batch is in the stash at a time per task. Memory footprint matches the current approach. + +6. **RangePartitioning sampling**: For RangePartitioning, `prepareShuffleDependency` creates a + separate sampling RDD that calls `batch.rowIterator()`. This happens before the shuffle write, + on a separate job. The sampling path does not go through `CometShuffleWriterInputIterator`. + No conflict. + +## Files Changed + +| File | Change | +|------|--------| +| `native/core/src/execution/batch_stash.rs` | **New**: BatchStash registry | +| `native/core/src/execution/mod.rs` | Export batch_stash module | +| `native/core/src/execution/jni_api.rs` | Add `executePlanBatchHandle` JNI function | +| `native/core/src/execution/operators/scan.rs` | Handle-based input path in ScanExec | +| `native/jni-bridge/src/batch_iterator.rs` | Add CometHandleBatchIterator JNI bridge struct | +| `spark/src/main/java/org/apache/comet/Native.java` | Add `executePlanBatchHandle` native method | +| `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` | **New**: handle passthrough iterator | +| `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` | Add stash mode + `nextHandle()` | +| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala` | Use CometShuffleWriterInputIterator in prepareShuffleDependency | +| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala` | Detect native input, use stash mode | + +## Testing + +1. **Rust unit test**: Test BatchStash -- stash, take, verify identity. Verify take returns None + for unknown handles. Verify take removes the entry. + +2. **JVM integration test**: Run a query that triggers CometNativeShuffle with a native child plan + (e.g., `SELECT * FROM t ORDER BY col`). Verify results match Spark. Verify the stash path was + taken (via metric or log). + +3. **Fallback test**: Run a query where the child is not CometNativeExec. Verify the FFI path + still works. + +4. **Regression**: Existing shuffle test suites (`CometShuffleSuite`, etc.) pass without + modification. From 488522cc4dbd807d02450a8e3ad85b32de0867d0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:09:02 -0600 Subject: [PATCH 02/21] docs: add implementation plan for batch stash shuffle optimization --- ...-04-12-batch-stash-shuffle-optimization.md | 1206 +++++++++++++++++ 1 file changed, 1206 insertions(+) create mode 100644 docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md diff --git a/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md b/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md new file mode 100644 index 0000000000..404fcefc1c --- /dev/null +++ b/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md @@ -0,0 +1,1206 @@ +# Batch Stash Shuffle Optimization Implementation Plan + +> **For agentic workers:** REQUIRED SUB-SKILL: Use superpowers:subagent-driven-development (recommended) or superpowers:executing-plans to implement this plan task-by-task. Steps use checkbox (`- [ ]`) syntax for tracking. + +**Goal:** Avoid unnecessary Arrow FFI import/export when passing batches between a native child plan and a native ShuffleWriter by using an opaque batch handle passed through the JVM. + +**Architecture:** A native-side `BatchStash` registry stores `RecordBatch` values keyed by `u64` handles. The child plan stashes its output batch and returns the handle to the JVM. The JVM passes the handle to the shuffle writer's `ScanExec`, which retrieves the batch directly from the stash. This eliminates 4 FFI boundary crossings per batch, replacing them with 2 lightweight JNI calls passing a single `long`. + +**Tech Stack:** Rust (native), Scala/Java (JVM), JNI, Arrow RecordBatch + +**Spec:** `docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md` + +--- + +### Task 1: BatchStash (Rust) + +**Files:** +- Create: `native/core/src/execution/batch_stash.rs` +- Modify: `native/core/src/execution/mod.rs:19-33` + +- [ ] **Step 1: Write the BatchStash test** + +Create `native/core/src/execution/batch_stash.rs` with a test module: + +```rust +// 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. + +//! A global registry for passing RecordBatch values between native execution +//! contexts through the JVM without Arrow FFI serialization. + +use std::collections::HashMap; +use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::Mutex; + +use arrow::record_batch::RecordBatch; +use once_cell::sync::Lazy; + +static NEXT_HANDLE: AtomicU64 = AtomicU64::new(1); +static STASH: Lazy>> = Lazy::new(|| Mutex::new(HashMap::new())); + +/// Stash a RecordBatch and return a unique handle for later retrieval. +pub fn stash(batch: RecordBatch) -> u64 { + todo!() +} + +/// Remove and return the RecordBatch for the given handle. +/// Returns `None` if the handle is not found (already consumed or invalid). +pub fn take(handle: u64) -> Option { + todo!() +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use std::sync::Arc; + + fn make_batch(values: &[i32]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let array = Int32Array::from(values.to_vec()); + RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap() + } + + #[test] + fn test_stash_and_take() { + let batch = make_batch(&[1, 2, 3]); + let handle = stash(batch); + assert!(handle > 0); + + let retrieved = take(handle).expect("batch should be in stash"); + assert_eq!(retrieved.num_rows(), 3); + assert_eq!( + retrieved + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .values(), + &[1, 2, 3] + ); + } + + #[test] + fn test_take_removes_entry() { + let batch = make_batch(&[10, 20]); + let handle = stash(batch); + + assert!(take(handle).is_some()); + assert!(take(handle).is_none(), "second take should return None"); + } + + #[test] + fn test_take_unknown_handle() { + assert!(take(999_999_999).is_none()); + } + + #[test] + fn test_handles_are_unique() { + let h1 = stash(make_batch(&[1])); + let h2 = stash(make_batch(&[2])); + assert_ne!(h1, h2); + + // Clean up + take(h1); + take(h2); + } +} +``` + +- [ ] **Step 2: Run test to verify it fails** + +Run: `cd native && cargo test --lib execution::batch_stash` +Expected: FAIL with `not yet implemented` + +- [ ] **Step 3: Implement stash and take** + +Replace the `todo!()` bodies in `native/core/src/execution/batch_stash.rs`: + +```rust +pub fn stash(batch: RecordBatch) -> u64 { + let handle = NEXT_HANDLE.fetch_add(1, Ordering::Relaxed); + STASH.lock().unwrap().insert(handle, batch); + handle +} + +pub fn take(handle: u64) -> Option { + STASH.lock().unwrap().remove(&handle) +} +``` + +- [ ] **Step 4: Add the module to mod.rs** + +In `native/core/src/execution/mod.rs`, add after line 18 (before existing pub mod declarations): + +```rust +pub(crate) mod batch_stash; +``` + +- [ ] **Step 5: Run tests to verify they pass** + +Run: `cd native && cargo test --lib execution::batch_stash` +Expected: all 4 tests PASS + +- [ ] **Step 6: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 7: Commit** + +```bash +git add native/core/src/execution/batch_stash.rs native/core/src/execution/mod.rs +git commit -m "feat: add BatchStash registry for native batch handle passing" +``` + +--- + +### Task 2: CometHandleBatchIterator (Java + Rust JNI bridge) + +**Files:** +- Create: `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` +- Create: `native/jni-bridge/src/handle_batch_iterator.rs` +- Modify: `native/jni-bridge/src/lib.rs:181-289` + +- [ ] **Step 1: Create CometHandleBatchIterator Java class** + +Create `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java`: + +```java +/* + * 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.comet; + +/** + * Iterator that passes opaque native batch handles between two native execution contexts through + * the JVM. Used when a native child plan feeds directly into a native ShuffleWriter, avoiding + * Arrow FFI export/import overhead. + * + *

Called from native ScanExec via JNI. The source CometExecIterator must be in stash mode. + */ +public class CometHandleBatchIterator { + private final CometExecIterator source; + + public CometHandleBatchIterator(CometExecIterator source) { + this.source = source; + } + + /** + * Get the next batch handle from the source iterator. + * + * @return a native batch handle (positive long), or -1 if no more batches. + */ + public long nextHandle() { + return source.nextHandle(); + } +} +``` + +- [ ] **Step 2: Create the Rust JNI bridge struct** + +Create `native/jni-bridge/src/handle_batch_iterator.rs`: + +```rust +// 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. + +use jni::signature::Primitive; +use jni::{ + errors::Result as JniResult, + objects::{JClass, JMethodID}, + signature::ReturnType, + strings::JNIString, + Env, +}; + +/// A struct that holds JNI methods for the JVM `CometHandleBatchIterator` class. +#[allow(dead_code)] // we need to keep references to Java items to prevent GC +pub struct CometHandleBatchIterator<'a> { + pub class: JClass<'a>, + pub method_next_handle: JMethodID, + pub method_next_handle_ret: ReturnType, +} + +impl<'a> CometHandleBatchIterator<'a> { + pub const JVM_CLASS: &'static str = "org/apache/comet/CometHandleBatchIterator"; + + pub fn new(env: &mut Env<'a>) -> JniResult> { + let class = env.find_class(JNIString::new(Self::JVM_CLASS))?; + + Ok(CometHandleBatchIterator { + class, + method_next_handle: env.get_method_id( + JNIString::new(Self::JVM_CLASS), + jni::jni_str!("nextHandle"), + jni::jni_sig!("()J"), + )?, + method_next_handle_ret: ReturnType::Primitive(Primitive::Long), + }) + } +} +``` + +- [ ] **Step 3: Register the new class in JVMClasses** + +In `native/jni-bridge/src/lib.rs`, add the module declaration after line 184 (`mod shuffle_block_iterator;`): + +```rust +mod handle_batch_iterator; +``` + +Add the import after line 189 (`use shuffle_block_iterator::CometShuffleBlockIterator;`): + +```rust +use handle_batch_iterator::CometHandleBatchIterator; +``` + +Add the field to the `JVMClasses` struct after line 216 (`pub comet_shuffle_block_iterator: CometShuffleBlockIterator<'a>,`): + +```rust + /// The CometHandleBatchIterator class. Used for passing batch handles between native plans. + pub comet_handle_batch_iterator: CometHandleBatchIterator<'a>, +``` + +Add initialization in the `JVMClasses::init` method after line 288 (`comet_shuffle_block_iterator: CometShuffleBlockIterator::new(env).unwrap(),`): + +```rust + comet_handle_batch_iterator: CometHandleBatchIterator::new(env).unwrap(), +``` + +- [ ] **Step 4: Verify it compiles** + +Run: `cd native && cargo build` +Expected: compilation succeeds (the Java class must exist on the classpath; if building native-only fails, run `make` from the project root first) + +- [ ] **Step 5: Commit** + +```bash +git add spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java \ + native/jni-bridge/src/handle_batch_iterator.rs \ + native/jni-bridge/src/lib.rs +git commit -m "feat: add CometHandleBatchIterator Java class and JNI bridge" +``` + +--- + +### Task 3: executePlanBatchHandle JNI function + +**Files:** +- Modify: `spark/src/main/scala/org/apache/comet/Native.scala:91-96` +- Modify: `native/core/src/execution/jni_api.rs:550-830` + +- [ ] **Step 1: Add the native method declaration in Native.scala** + +In `spark/src/main/scala/org/apache/comet/Native.scala`, add after the `executePlan` method (after line 96): + +```scala + /** + * Execute one step of the native query plan, stashing the output RecordBatch in the native + * BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) + * or -1 for EOF. Used when the output feeds directly into another native plan (e.g., + * native ShuffleWriter) to avoid unnecessary FFI round-trips. + * + * @param stage + * the stage ID, for informational purposes + * @param partition + * the partition ID, for informational purposes + * @param plan + * the address to native query plan. + * @return + * a batch stash handle (positive), or -1 for EOF. + */ + @native def executePlanBatchHandle( + stage: Int, + partition: Int, + plan: Long): Long +``` + +- [ ] **Step 2: Add the stash_output helper function in jni_api.rs** + +In `native/core/src/execution/jni_api.rs`, add after the `prepare_output` function (after line 618): + +```rust +/// Stash the output RecordBatch in the BatchStash and return the handle. +/// Used when output feeds directly into another native plan. +fn stash_output(output_batch: RecordBatch) -> CometResult { + let handle = crate::execution::batch_stash::stash(output_batch); + Ok(handle as jlong) +} +``` + +- [ ] **Step 3: Add the JNI function implementation** + +In `native/core/src/execution/jni_api.rs`, add after the `Java_org_apache_comet_Native_executePlan` function (after line 830): + +```rust +/// Like executePlan but stashes the output RecordBatch in the BatchStash and returns +/// the handle instead of exporting via Arrow FFI. Used for native-to-native batch passing. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlanBatchHandle( + e: EnvUnowned, + _class: JClass, + stage_id: jint, + partition: jint, + exec_context: jlong, +) -> jlong { + try_unwrap_or_throw(&e, |env| { + let exec_context = get_execution_context(exec_context); + + let tracing_enabled = exec_context.tracing_enabled; + let owned_label; + let tracing_label = if tracing_enabled { + owned_label = exec_context.tracing_event_name.clone(); + owned_label.as_str() + } else { + "" + }; + + let result = with_trace(tracing_label, tracing_enabled, || { + let exec_context_id = exec_context.id; + + // Initialize the execution stream on first call (same as executePlan) + if exec_context.root_op.is_none() { + let start = Instant::now(); + let planner = + PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) + .with_exec_id(exec_context_id); + let (scans, shuffle_scans, root_op) = planner.create_plan( + &exec_context.spark_plan, + &mut exec_context.input_sources.clone(), + exec_context.partition_count, + )?; + let physical_plan_time = start.elapsed(); + + exec_context.plan_creation_time += physical_plan_time; + exec_context.scans = scans; + exec_context.shuffle_scans = shuffle_scans; + + if exec_context.explain_native { + let formatted_plan_str = + DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); + info!("Comet native query plan:\n{formatted_plan_str:}"); + } + + let task_ctx = exec_context.session_ctx.task_ctx(); + let stream = root_op.native_plan.execute(0, task_ctx)?; + + if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { + let (tx, rx) = mpsc::channel(2); + let mut stream = stream; + get_runtime().spawn(async move { + let result = std::panic::AssertUnwindSafe(async { + while let Some(batch) = stream.next().await { + if tx.send(batch).await.is_err() { + break; + } + } + }) + .catch_unwind() + .await; + + if let Err(panic) = result { + let msg = match panic.downcast_ref::<&str>() { + Some(s) => s.to_string(), + None => match panic.downcast_ref::() { + Some(s) => s.clone(), + None => "unknown panic".to_string(), + }, + }; + let _ = tx + .send(Err(DataFusionError::Execution(format!( + "native panic: {msg}" + )))) + .await; + } + }); + exec_context.batch_receiver = Some(rx); + } else { + exec_context.stream = Some(stream); + } + exec_context.root_op = Some(root_op); + } else { + pull_input_batches(exec_context)?; + } + + if let Some(rx) = &mut exec_context.batch_receiver { + match rx.blocking_recv() { + Some(Ok(batch)) => { + update_metrics(env, exec_context)?; + return stash_output(batch); + } + Some(Err(e)) => { + return Err(e.into()); + } + None => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + } + } + + // ScanExec path: busy-poll + get_runtime().block_on(async { + loop { + let next_item = exec_context.stream.as_mut().unwrap().next(); + let poll_output = poll!(next_item); + + exec_context.poll_count_since_metrics_check += 1; + if exec_context.poll_count_since_metrics_check >= 100 { + exec_context.poll_count_since_metrics_check = 0; + if let Some(interval) = exec_context.metrics_update_interval { + let now = Instant::now(); + if now - exec_context.metrics_last_update_time >= interval { + update_metrics(env, exec_context)?; + exec_context.metrics_last_update_time = now; + } + } + if exec_context.tracing_enabled { + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + } + + match poll_output { + Poll::Ready(Some(output)) => { + return stash_output(output?); + } + Poll::Ready(None) => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + Poll::Pending => { + tokio::task::block_in_place(|| pull_input_batches(exec_context))?; + } + } + } + }) + }); + + if exec_context.tracing_enabled { + #[cfg(feature = "jemalloc")] + log_jemalloc_usage(); + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + + result + }) +} +``` + +- [ ] **Step 4: Verify it compiles** + +Run: `cd native && cargo build` +Expected: compilation succeeds + +- [ ] **Step 5: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 6: Commit** + +```bash +git add spark/src/main/scala/org/apache/comet/Native.scala \ + native/core/src/execution/jni_api.rs +git commit -m "feat: add executePlanBatchHandle JNI function for stash-mode output" +``` + +--- + +### Task 4: CometExecIterator stash mode + +**Files:** +- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` + +- [ ] **Step 1: Add stash mode fields** + +In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add after line 81 (`private val cometTaskMemoryManager = new CometTaskMemoryManager(id, taskAttemptId)`): + +```scala + // When true, executePlan stashes output batches natively and returns handles + // instead of exporting via Arrow FFI. Used when output feeds a native ShuffleWriter. + @volatile private var stashMode: Boolean = false + private var pendingHandle: Long = -1L +``` + +- [ ] **Step 2: Add enableStashMode and nextHandle methods** + +In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add after the `close()` method (after line 254, before the `traceMemoryUsage` method): + +```scala + /** Enable stash mode. Must be called before iteration begins. */ + def enableStashMode(): Unit = { + stashMode = true + } + + /** + * In stash mode, advance the native plan and return the batch handle. + * Returns a positive handle, or -1 for EOF. + */ + def nextHandle(): Long = { + if (closed) return -1L + + if (pendingHandle >= 0) { + val h = pendingHandle + pendingHandle = -1L + return h + } + + val ctx = TaskContext.get() + try { + val handle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) + if (handle == -1L) { + close() + } + handle + } catch { + case e: Throwable => throw e + } + } +``` + +- [ ] **Step 3: Modify hasNext to support stash mode** + +In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, replace the `hasNext` method (lines 189-214) with: + +```scala + override def hasNext: Boolean = { + if (closed) return false + + if (stashMode) { + // In stash mode, we use nextHandle() instead of getNextBatch. + // hasNext is called by the shuffle writer's CometExecIterator indirectly. + // We probe for the next handle and cache it. + if (pendingHandle >= 0) return true + val ctx = TaskContext.get() + pendingHandle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) + if (pendingHandle == -1L) { + close() + false + } else { + true + } + } else { + if (nextBatch.isDefined) { + return true + } + + if (prevBatch != null) { + prevBatch.close() + prevBatch = null + } + + nextBatch = getNextBatch + + logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") + + if (nextBatch.isEmpty) { + close() + false + } else { + true + } + } + } +``` + +- [ ] **Step 4: Modify next to handle stash mode** + +In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, replace the `next()` method (lines 216-231) with: + +```scala + override def next(): ColumnarBatch = { + if (stashMode) { + // In stash mode, next() should not be called directly. + // The shuffle writer uses nextHandle() instead. + throw new UnsupportedOperationException( + "next() should not be called in stash mode. Use nextHandle() instead.") + } + + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + + if (nextBatch.isEmpty && !hasNext) { + throw new NoSuchElementException("No more element") + } + + currentBatch = nextBatch.get + prevBatch = currentBatch + nextBatch = None + currentBatch + } +``` + +- [ ] **Step 5: Build to verify compilation** + +Run: `make` +Expected: build succeeds + +- [ ] **Step 6: Commit** + +```bash +git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala +git commit -m "feat: add stash mode to CometExecIterator for batch handle output" +``` + +--- + +### Task 5: ScanExec handle-based input path + +**Files:** +- Modify: `native/core/src/execution/operators/scan.rs:57-258` + +- [ ] **Step 1: Add handle_mode flag to ScanExec** + +In `native/core/src/execution/operators/scan.rs`, add a new field to the `ScanExec` struct after line 79 (`arrow_ffi_safe: bool,`): + +```rust + /// When true, input comes from a CometHandleBatchIterator and batches are + /// retrieved from the BatchStash instead of via Arrow FFI import. + pub handle_mode: bool, +``` + +Update the `ScanExec::new` method to add `handle_mode: false` in the `Ok(Self { ... })` block (after the `arrow_ffi_safe,` line around line 115): + +```rust + handle_mode: false, +``` + +- [ ] **Step 2: Add get_next_handle method** + +In `native/core/src/execution/operators/scan.rs`, add after the `get_next` method (after line 258): + +```rust + /// Pull next input batch from a CometHandleBatchIterator via batch stash handle. + fn get_next_handle( + exec_context_id: i64, + iter: &JObject, + ) -> Result { + if exec_context_id == TEST_EXEC_CONTEXT_ID { + return Ok(InputBatch::EOF); + } + + if iter.is_null() { + return Err(CometError::from(ExecutionError::GeneralError(format!( + "Null handle batch iterator object. Plan id: {exec_context_id}" + )))); + } + + JVMClasses::with_env(|env| { + let handle: i64 = unsafe { + jni_call!(env, + comet_handle_batch_iterator(iter).next_handle() -> i64)? + }; + + if handle == -1 { + return Ok(InputBatch::EOF); + } + + match crate::execution::batch_stash::take(handle as u64) { + Some(batch) => { + let arrays: Vec = batch.columns().to_vec(); + let num_rows = batch.num_rows(); + Ok(InputBatch::new(arrays, Some(num_rows))) + } + None => Err(CometError::from(ExecutionError::GeneralError(format!( + "Batch stash handle {handle} not found" + )))), + } + }) + } +``` + +- [ ] **Step 3: Modify get_next_batch to use handle mode** + +In `native/core/src/execution/operators/scan.rs`, replace the `get_next_batch` method (lines 135-156) with: + +```rust + /// Pull next input batch from JVM. + pub fn get_next_batch(&mut self) -> Result<(), CometError> { + if self.input_source.is_none() { + // This is a unit test. We don't need to call JNI. + return Ok(()); + } + let mut timer = self.baseline_metrics.elapsed_compute().timer(); + + let mut current_batch = self.batch.try_lock().unwrap(); + if current_batch.is_none() { + let next_batch = if self.handle_mode { + ScanExec::get_next_handle( + self.exec_context_id, + self.input_source.as_ref().unwrap().as_obj(), + )? + } else { + ScanExec::get_next( + self.exec_context_id, + self.input_source.as_ref().unwrap().as_obj(), + self.data_types.len(), + self.arrow_ffi_safe, + )? + }; + *current_batch = Some(next_batch); + } + + timer.stop(); + + Ok(()) + } +``` + +- [ ] **Step 4: Verify it compiles** + +Run: `cd native && cargo build` +Expected: compilation succeeds + +- [ ] **Step 5: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 6: Commit** + +```bash +git add native/core/src/execution/operators/scan.rs +git commit -m "feat: add handle-mode input path to ScanExec for batch stash retrieval" +``` + +--- + +### Task 6: Planner detection of CometHandleBatchIterator + +**Files:** +- Modify: `native/core/src/execution/planner.rs:1298-1311` + +- [ ] **Step 1: Add runtime class check in planner** + +In `native/core/src/execution/planner.rs`, replace the Scan creation block (lines 1298-1311) with: + +```rust + // The `ScanExec` operator will take actual arrays from Spark during execution + let mut scan = ScanExec::new( + self.exec_context_id, + input_source.clone(), + &scan.source, + data_types, + scan.arrow_ffi_safe, + )?; + + // Check if the input source is a CometHandleBatchIterator. + // If so, enable handle mode on the scan to retrieve batches from + // the BatchStash instead of via Arrow FFI. + if let Some(ref source) = input_source { + let is_handle_iter = JVMClasses::with_env(|env| { + let handle_class = + &JVMClasses::get().comet_handle_batch_iterator.class; + let result = env.is_instance_of(source.as_obj(), handle_class)?; + Ok::(result) + })?; + if is_handle_iter { + scan.handle_mode = true; + } + } + + Ok(( + vec![scan.clone()], + vec![], + Arc::new(SparkPlan::new(spark_plan.plan_id, Arc::new(scan), vec![])), + )) +``` + +- [ ] **Step 2: Add the necessary import** + +In `native/core/src/execution/planner.rs`, check if `JVMClasses` and `CometError` are already imported. If `JVMClasses` is not imported, add to the imports at the top of the file: + +```rust +use datafusion_comet_jni_bridge::JVMClasses; +``` + +(`CometError` should already be imported via the existing error handling imports.) + +- [ ] **Step 3: Verify it compiles** + +Run: `cd native && cargo build` +Expected: compilation succeeds + +- [ ] **Step 4: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 5: Commit** + +```bash +git add native/core/src/execution/planner.rs +git commit -m "feat: detect CometHandleBatchIterator in planner and enable handle mode" +``` + +--- + +### Task 7: CometShuffleWriterInputIterator and detection in prepareShuffleDependency + +**Files:** +- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala:584-658` + +- [ ] **Step 1: Add CometShuffleWriterInputIterator class** + +In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, add at the end of the file (before the final closing brace of the companion object, or after it if it's a top-level addition): + +```scala +/** + * An iterator wrapper that preserves access to the underlying CometExecIterator + * when present. Used by CometNativeShuffleWriter to detect native child plans + * and enable the batch stash optimization. + */ +private[shuffle] class CometShuffleWriterInputIterator( + underlying: Iterator[ColumnarBatch], + val nativeIterator: Option[CometExecIterator]) + extends Iterator[Product2[Int, ColumnarBatch]] { + override def hasNext: Boolean = underlying.hasNext + override def next(): Product2[Int, ColumnarBatch] = (0, underlying.next()) +} +``` + +- [ ] **Step 2: Add the necessary import** + +In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, add to the imports: + +```scala +import org.apache.comet.CometExecIterator +``` + +- [ ] **Step 3: Modify prepareShuffleDependency to use CometShuffleWriterInputIterator** + +In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, replace lines 643-646: + +```scala + val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( + rdd.map( + (0, _) + ), // adding fake partitionId that is always 0 because ShuffleDependency requires it +``` + +with: + +```scala + val wrappedRDD = rdd.mapPartitions { iter => + val nativeIter = iter match { + case cei: CometExecIterator => Some(cei) + case _ => None + } + new CometShuffleWriterInputIterator(iter, nativeIter) + } + + val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( + wrappedRDD, +``` + +- [ ] **Step 4: Build to verify compilation** + +Run: `make` +Expected: build succeeds + +- [ ] **Step 5: Commit** + +```bash +git add spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +git commit -m "feat: preserve CometExecIterator reference through shuffle dependency RDD" +``` + +--- + +### Task 8: CometNativeShuffleWriter stash mode integration + +**Files:** +- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala:65-112` +- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala:84-89` + +- [ ] **Step 1: Add a CometExecIterator constructor that accepts pre-built inputIterators** + +In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add a secondary constructor or a new `getCometIterator` overload. The simplest approach is to add a new parameter to accept pre-built input iterators. + +Add a new companion-object-style factory method. In `spark/src/main/scala/org/apache/spark/sql/comet/operators.scala`, add a new `getCometIterator` overload after line 371 (after the existing overloads): + +```scala + /** + * Create a CometExecIterator with pre-built input iterators (e.g., CometHandleBatchIterator). + * Bypasses the normal CometBatchIterator wrapping. + */ + def getCometIteratorWithHandleInputs( + handleInputs: Array[Object], + numOutputCols: Int, + nativePlan: Operator, + nativeMetrics: CometMetricNode, + numParts: Int, + partitionIdx: Int): CometExecIterator = { + val bytes = serializeNativePlan(nativePlan) + new CometExecIterator( + newIterId, + Seq.empty, // no ColumnarBatch inputs + numOutputCols, + bytes, + nativeMetrics, + numParts, + partitionIdx, + handleInputs = handleInputs) + } +``` + +Then in `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add an optional `handleInputs` parameter to the constructor. Modify the class declaration (line 61) to add the parameter: + +```scala +class CometExecIterator( + val id: Long, + inputs: Seq[Iterator[ColumnarBatch]], + numOutputCols: Int, + protobufQueryPlan: Array[Byte], + nativeMetrics: CometMetricNode, + numParts: Int, + partitionIndex: Int, + broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, + encryptedFilePaths: Seq[String] = Seq.empty, + shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty, + handleInputs: Array[Object] = Array.empty) +``` + +Then modify the `inputIterators` initialization (lines 84-89) to prefer handleInputs when provided: + +```scala + private val inputIterators: Array[Object] = if (handleInputs.nonEmpty) { + handleInputs + } else { + inputs.zipWithIndex.map { + case (_, idx) if shuffleBlockIterators.contains(idx) => + shuffleBlockIterators(idx).asInstanceOf[Object] + case (iterator, _) => + new CometBatchIterator(iterator, nativeUtil).asInstanceOf[Object] + }.toArray + } +``` + +- [ ] **Step 2: Modify CometNativeShuffleWriter.write() to detect and use stash mode** + +In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala`, add the import at the top: + +```scala +import org.apache.comet.{CometExecIterator, CometHandleBatchIterator} +``` + +Replace lines 96-111 (from `val newInputs` through the `while` loop) with: + +```scala + // Detect if input comes from a native plan (CometExecIterator) + val nativeIter: Option[CometExecIterator] = inputs match { + case swi: CometShuffleWriterInputIterator => swi.nativeIterator + case _ => None + } + + val cometIter = nativeIter match { + case Some(childIter) => + // Stash mode: child plan stashes batches, shuffle writer retrieves via handles + childIter.enableStashMode() + val handleIter = new CometHandleBatchIterator(childIter) + CometExec.getCometIteratorWithHandleInputs( + Array(handleIter.asInstanceOf[Object]), + outputAttributes.length, + nativePlan, + nativeMetrics, + numParts, + context.partitionId()) + case None => + // Normal FFI mode: wrap input in CometBatchIterator as before + val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) + CometExec.getCometIterator( + Seq(newInputs.asInstanceOf[Iterator[ColumnarBatch]]), + outputAttributes.length, + nativePlan, + nativeMetrics, + numParts, + context.partitionId(), + broadcastedHadoopConfForEncryption = None, + encryptedFilePaths = Seq.empty) + } + + while (cometIter.hasNext) { + cometIter.next() + } + cometIter.close() +``` + +- [ ] **Step 3: Add the import for CometShuffleWriterInputIterator** + +The `CometShuffleWriterInputIterator` is in the same package (`execution.shuffle`), so it should be accessible without an explicit import. Verify this compiles. + +- [ ] **Step 4: Build to verify compilation** + +Run: `make` +Expected: build succeeds + +- [ ] **Step 5: Commit** + +```bash +git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala \ + spark/src/main/scala/org/apache/spark/sql/comet/operators.scala \ + spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +git commit -m "feat: integrate batch stash mode in CometNativeShuffleWriter" +``` + +--- + +### Task 9: End-to-end testing + +**Files:** +- Modify: existing shuffle test suite (no new test files needed) + +- [ ] **Step 1: Run existing shuffle tests to verify no regressions** + +Run: `./mvnw test -DwildcardSuites="CometShuffleSuite" -Dtest=none` +Expected: all tests PASS + +- [ ] **Step 2: Run the full native shuffle test suite** + +Run: `./mvnw test -DwildcardSuites="CometNativeShuffleSuite" -Dtest=none` +Expected: all tests PASS (if this suite exists; otherwise skip) + +- [ ] **Step 3: Run a broader test to cover shuffle exchange paths** + +Run: `./mvnw test -DwildcardSuites="CometExec" -Dtest=none` +Expected: all tests PASS + +- [ ] **Step 4: Run Rust tests** + +Run: `cd native && cargo test --workspace` +Expected: all tests PASS (including the BatchStash tests from Task 1) + +- [ ] **Step 5: Run clippy one final time** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 6: Format all code** + +Run: `make format` +Expected: formatting applied cleanly + +- [ ] **Step 7: Commit any formatting changes** + +```bash +git add -A +git commit -m "style: format code" +``` + +--- + +### Task 10: Refactor to reduce duplication in jni_api.rs + +The `executePlanBatchHandle` JNI function duplicates most of `executePlan`. After verifying correctness in Task 9, extract the shared logic. + +**Files:** +- Modify: `native/core/src/execution/jni_api.rs` + +- [ ] **Step 1: Extract shared execution logic into a helper** + +In `native/core/src/execution/jni_api.rs`, create a helper enum and function: + +```rust +/// How to handle the output batch from executePlan. +enum OutputMode<'a> { + /// Export via Arrow FFI to the provided addresses. + Ffi { + env: &'a mut Env<'a>, + array_addrs: JLongArray<'a>, + schema_addrs: JLongArray<'a>, + validate: bool, + }, + /// Stash in BatchStash and return handle. + Stash, +} +``` + +Then extract the common body of `executePlan` and `executePlanBatchHandle` into: + +```rust +fn execute_plan_inner( + env: &mut Env, + exec_context: &mut ExecutionContext, + stage_id: jint, + partition: jint, + output_mode: OutputMode, +) -> CometResult +``` + +This function contains the shared initialization, stream polling, and metrics logic. The only difference is the final step: `OutputMode::Ffi` calls `prepare_output()`, `OutputMode::Stash` calls `stash_output()`. + +Both `Java_org_apache_comet_Native_executePlan` and `Java_org_apache_comet_Native_executePlanBatchHandle` become thin wrappers that call `execute_plan_inner` with the appropriate `OutputMode`. + +- [ ] **Step 2: Verify it compiles and tests pass** + +Run: `cd native && cargo build && cargo test --workspace` +Expected: compilation succeeds, all tests pass + +- [ ] **Step 3: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: no warnings + +- [ ] **Step 4: Commit** + +```bash +git add native/core/src/execution/jni_api.rs +git commit -m "refactor: extract shared execution logic from executePlan and executePlanBatchHandle" +``` From ca8ee6a6ff1ee93d1b993bf21f13ff4c1a7a642c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:13:28 -0600 Subject: [PATCH 03/21] feat: add BatchStash registry for native batch handle passing --- native/core/src/execution/batch_stash.rs | 125 +++++++++++++++++++++++ native/core/src/execution/mod.rs | 1 + 2 files changed, 126 insertions(+) create mode 100644 native/core/src/execution/batch_stash.rs diff --git a/native/core/src/execution/batch_stash.rs b/native/core/src/execution/batch_stash.rs new file mode 100644 index 0000000000..a5eaad42cc --- /dev/null +++ b/native/core/src/execution/batch_stash.rs @@ -0,0 +1,125 @@ +// 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. + +//! Global registry for passing RecordBatch values between native execution contexts +//! via opaque u64 handles, without Arrow FFI serialization. + +// These items will be called from JNI and other modules added in subsequent tasks. +#![allow(dead_code)] + +use arrow::record_batch::RecordBatch; +use once_cell::sync::Lazy; +use std::collections::HashMap; +use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::Mutex; + +/// Counter for generating unique handles. +static NEXT_HANDLE: AtomicU64 = AtomicU64::new(1); + +/// Global stash mapping handles to RecordBatch values. +static STASH: Lazy>> = + Lazy::new(|| Mutex::new(HashMap::new())); + +/// Store a RecordBatch in the global stash and return a unique handle. +pub(crate) fn stash(batch: RecordBatch) -> u64 { + let handle = NEXT_HANDLE.fetch_add(1, Ordering::Relaxed); + STASH + .lock() + .expect("batch_stash lock poisoned") + .insert(handle, batch); + handle +} + +/// Remove and return the RecordBatch associated with the given handle. +/// +/// Returns `None` if the handle does not exist in the stash. +pub(crate) fn take(handle: u64) -> Option { + STASH + .lock() + .expect("batch_stash lock poisoned") + .remove(&handle) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use std::sync::Arc; + + fn make_batch(values: Vec) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let array = Arc::new(Int32Array::from(values)); + RecordBatch::try_new(schema, vec![array]).unwrap() + } + + #[test] + fn test_stash_and_take() { + let batch = make_batch(vec![1, 2, 3]); + let num_rows = batch.num_rows(); + + let handle = stash(batch); + let retrieved = take(handle).expect("expected batch to be present"); + + assert_eq!(retrieved.num_rows(), num_rows); + let col = retrieved + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.values(), &[1, 2, 3]); + } + + #[test] + fn test_take_removes_entry() { + let batch = make_batch(vec![10, 20]); + let handle = stash(batch); + + // First take returns the batch. + assert!(take(handle).is_some()); + // Second take finds nothing. + assert!(take(handle).is_none()); + } + + #[test] + fn test_take_unknown_handle() { + // Handle 0 is never issued (counter starts at 1). + assert!(take(0).is_none()); + // A large handle that was never issued. + assert!(take(u64::MAX).is_none()); + } + + #[test] + fn test_handles_are_unique() { + let batch1 = make_batch(vec![1]); + let batch2 = make_batch(vec![2]); + let batch3 = make_batch(vec![3]); + + let h1 = stash(batch1); + let h2 = stash(batch2); + let h3 = stash(batch3); + + assert_ne!(h1, h2); + assert_ne!(h2, h3); + assert_ne!(h1, h3); + + // Clean up. + take(h1); + take(h2); + take(h3); + } +} diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index f556fce41c..b25fe277ff 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -16,6 +16,7 @@ // under the License. //! PoC of vectorization execution through JNI to Rust. +pub(crate) mod batch_stash; pub mod columnar_to_row; pub mod expressions; pub mod jni_api; From e68d7777277712e3504677667c0ad1f2fdda15df Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:14:57 -0600 Subject: [PATCH 04/21] feat: add CometHandleBatchIterator Java class and JNI bridge --- .../jni-bridge/src/handle_batch_iterator.rs | 51 +++++++++++++++++++ native/jni-bridge/src/lib.rs | 5 ++ .../comet/CometHandleBatchIterator.java | 44 ++++++++++++++++ 3 files changed, 100 insertions(+) create mode 100644 native/jni-bridge/src/handle_batch_iterator.rs create mode 100644 spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java diff --git a/native/jni-bridge/src/handle_batch_iterator.rs b/native/jni-bridge/src/handle_batch_iterator.rs new file mode 100644 index 0000000000..b37637ce0b --- /dev/null +++ b/native/jni-bridge/src/handle_batch_iterator.rs @@ -0,0 +1,51 @@ +// 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. + +use jni::signature::Primitive; +use jni::{ + errors::Result as JniResult, + objects::{JClass, JMethodID}, + signature::ReturnType, + strings::JNIString, + Env, +}; + +/// A struct that holds JNI methods for the JVM `CometHandleBatchIterator` class. +#[allow(dead_code)] // we need to keep references to Java items to prevent GC +pub struct CometHandleBatchIterator<'a> { + pub class: JClass<'a>, + pub method_next_handle: JMethodID, + pub method_next_handle_ret: ReturnType, +} + +impl<'a> CometHandleBatchIterator<'a> { + pub const JVM_CLASS: &'static str = "org/apache/comet/CometHandleBatchIterator"; + + pub fn new(env: &mut Env<'a>) -> JniResult> { + let class = env.find_class(JNIString::new(Self::JVM_CLASS))?; + + Ok(CometHandleBatchIterator { + class, + method_next_handle: env.get_method_id( + JNIString::new(Self::JVM_CLASS), + jni::jni_str!("nextHandle"), + jni::jni_sig!("()J"), + )?, + method_next_handle_ret: ReturnType::Primitive(Primitive::Long), + }) + } +} diff --git a/native/jni-bridge/src/lib.rs b/native/jni-bridge/src/lib.rs index 5b0c0a4a56..82b85d4be4 100644 --- a/native/jni-bridge/src/lib.rs +++ b/native/jni-bridge/src/lib.rs @@ -181,11 +181,13 @@ pub use comet_exec::*; mod batch_iterator; mod comet_metric_node; mod comet_task_memory_manager; +mod handle_batch_iterator; mod shuffle_block_iterator; use batch_iterator::CometBatchIterator; pub use comet_metric_node::*; pub use comet_task_memory_manager::*; +use handle_batch_iterator::CometHandleBatchIterator; use shuffle_block_iterator::CometShuffleBlockIterator; /// The JVM classes that are used in the JNI calls. @@ -214,6 +216,8 @@ pub struct JVMClasses<'a> { pub comet_batch_iterator: CometBatchIterator<'a>, /// The CometShuffleBlockIterator class. Used for iterating over shuffle blocks. pub comet_shuffle_block_iterator: CometShuffleBlockIterator<'a>, + /// The CometHandleBatchIterator class. Used for passing batch handles between native contexts. + pub comet_handle_batch_iterator: CometHandleBatchIterator<'a>, /// The CometTaskMemoryManager used for interacting with JVM side to /// acquire & release native memory. pub comet_task_memory_manager: CometTaskMemoryManager<'a>, @@ -286,6 +290,7 @@ impl JVMClasses<'_> { comet_exec: CometExec::new(env).unwrap(), comet_batch_iterator: CometBatchIterator::new(env).unwrap(), comet_shuffle_block_iterator: CometShuffleBlockIterator::new(env).unwrap(), + comet_handle_batch_iterator: CometHandleBatchIterator::new(env).unwrap(), comet_task_memory_manager: CometTaskMemoryManager::new(env).unwrap(), } }); diff --git a/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java b/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java new file mode 100644 index 0000000000..dbc43f7d71 --- /dev/null +++ b/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet; + +/** + * Iterator that passes opaque native batch handles between two native execution contexts through + * the JVM. Used when a native child plan feeds directly into a native ShuffleWriter, avoiding + * Arrow FFI export/import overhead. + * + *

Called from native ScanExec via JNI. The source CometExecIterator must be in stash mode. + */ +public class CometHandleBatchIterator { + private final CometExecIterator source; + + public CometHandleBatchIterator(CometExecIterator source) { + this.source = source; + } + + /** + * Get the next batch handle from the source iterator. + * + * @return a native batch handle (positive long), or -1 if no more batches. + */ + public long nextHandle() { + return source.nextHandle(); + } +} From 07c86c76cfc4b68dd10598b7a542c28d5efbd37e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:16:48 -0600 Subject: [PATCH 05/21] feat: add executePlanBatchHandle JNI function for stash-mode output --- native/core/src/execution/jni_api.rs | 188 ++++++++++++++++++ .../main/scala/org/apache/comet/Native.scala | 20 ++ 2 files changed, 208 insertions(+) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 93f75bae96..ae70c14e0d 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -617,6 +617,13 @@ fn prepare_output( Ok(num_rows as jlong) } +/// Stash the output RecordBatch in the BatchStash and return the handle. +/// Used when output feeds directly into another native plan. +fn stash_output(output_batch: RecordBatch) -> CometResult { + let handle = crate::execution::batch_stash::stash(output_batch); + Ok(handle as jlong) +} + /// Pull the next input from JVM. Note that we cannot pull input batches in /// `ScanStream.poll_next` when the execution stream is polled for output. /// Because the input source could be another native execution stream, which @@ -829,6 +836,187 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( }) } +/// Execute one step of the native query plan, stashing the output RecordBatch in the +/// BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) +/// or -1 for EOF. Used when the output feeds directly into another native plan. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlanBatchHandle( + e: EnvUnowned, + _class: JClass, + stage_id: jint, + partition: jint, + exec_context: jlong, +) -> jlong { + try_unwrap_or_throw(&e, |env| { + // Retrieve the query + let exec_context = get_execution_context(exec_context); + + let tracing_enabled = exec_context.tracing_enabled; + // Clone the label only when tracing is enabled. The clone is needed + // because the closure below mutably borrows exec_context. + let owned_label; + let tracing_label = if tracing_enabled { + owned_label = exec_context.tracing_event_name.clone(); + owned_label.as_str() + } else { + "" + }; + + let result = with_trace(tracing_label, tracing_enabled, || { + let exec_context_id = exec_context.id; + + // Initialize the execution stream. + // Because we don't know if input arrays are dictionary-encoded when we create + // query plan, we need to defer stream initialization to first time execution. + if exec_context.root_op.is_none() { + let start = Instant::now(); + let planner = + PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) + .with_exec_id(exec_context_id); + let (scans, shuffle_scans, root_op) = planner.create_plan( + &exec_context.spark_plan, + &mut exec_context.input_sources.clone(), + exec_context.partition_count, + )?; + let physical_plan_time = start.elapsed(); + + exec_context.plan_creation_time += physical_plan_time; + exec_context.scans = scans; + exec_context.shuffle_scans = shuffle_scans; + + if exec_context.explain_native { + let formatted_plan_str = + DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); + info!("Comet native query plan:\n{formatted_plan_str:}"); + } + + let task_ctx = exec_context.session_ctx.task_ctx(); + // Each Comet native execution corresponds to a single Spark partition, + // so we should always execute partition 0. + let stream = root_op.native_plan.execute(0, task_ctx)?; + + if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { + // No JVM data sources — spawn onto tokio so the executor + // thread parks in blocking_recv instead of busy-polling. + // + // Channel capacity of 2 allows the producer to work one batch + // ahead while the consumer processes the current one via JNI, + // without buffering excessive memory. Increasing this would + // trade memory for latency hiding if JNI/FFI overhead dominates; + // decreasing to 1 would serialize production and consumption. + let (tx, rx) = mpsc::channel(2); + let mut stream = stream; + get_runtime().spawn(async move { + let result = std::panic::AssertUnwindSafe(async { + while let Some(batch) = stream.next().await { + if tx.send(batch).await.is_err() { + break; + } + } + }) + .catch_unwind() + .await; + + if let Err(panic) = result { + let msg = match panic.downcast_ref::<&str>() { + Some(s) => s.to_string(), + None => match panic.downcast_ref::() { + Some(s) => s.clone(), + None => "unknown panic".to_string(), + }, + }; + let _ = tx + .send(Err(DataFusionError::Execution(format!( + "native panic: {msg}" + )))) + .await; + } + }); + exec_context.batch_receiver = Some(rx); + } else { + exec_context.stream = Some(stream); + } + exec_context.root_op = Some(root_op); + } else { + // Pull input batches + pull_input_batches(exec_context)?; + } + + if let Some(rx) = &mut exec_context.batch_receiver { + match rx.blocking_recv() { + Some(Ok(batch)) => { + update_metrics(env, exec_context)?; + return stash_output(batch); + } + Some(Err(e)) => { + return Err(e.into()); + } + None => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + } + } + + // ScanExec path: busy-poll to interleave JVM batch pulls with stream polling + get_runtime().block_on(async { + loop { + let next_item = exec_context.stream.as_mut().unwrap().next(); + let poll_output = poll!(next_item); + + // Only check time/tracing every 100 polls to reduce overhead + exec_context.poll_count_since_metrics_check += 1; + if exec_context.poll_count_since_metrics_check >= 100 { + exec_context.poll_count_since_metrics_check = 0; + if let Some(interval) = exec_context.metrics_update_interval { + let now = Instant::now(); + if now - exec_context.metrics_last_update_time >= interval { + update_metrics(env, exec_context)?; + exec_context.metrics_last_update_time = now; + } + } + if exec_context.tracing_enabled { + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + } + + match poll_output { + Poll::Ready(Some(output)) => { + return stash_output(output?); + } + Poll::Ready(None) => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + Poll::Pending => { + // JNI call to pull batches from JVM into ScanExec operators. + // block_in_place lets tokio move other tasks off this worker + // while we wait for JVM data. + tokio::task::block_in_place(|| pull_input_batches(exec_context))?; + } + } + } + }) + }); + + if exec_context.tracing_enabled { + #[cfg(feature = "jemalloc")] + log_jemalloc_usage(); + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + + result + }) +} + #[no_mangle] /// Drop the native query plan object and context object. pub extern "system" fn Java_org_apache_comet_Native_releasePlan( diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index c003bcd138..55d0555970 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -95,6 +95,26 @@ class Native extends NativeBase { arrayAddrs: Array[Long], schemaAddrs: Array[Long]): Long + /** + * Execute one step of the native query plan, stashing the output RecordBatch in the native + * BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) + * or -1 for EOF. Used when the output feeds directly into another native plan (e.g., + * native ShuffleWriter) to avoid unnecessary FFI round-trips. + * + * @param stage + * the stage ID, for informational purposes + * @param partition + * the partition ID, for informational purposes + * @param plan + * the address to native query plan. + * @return + * a batch stash handle (positive), or -1 for EOF. + */ + @native def executePlanBatchHandle( + stage: Int, + partition: Int, + plan: Long): Long + /** * Release and drop the native query plan object and context object. * From d2f87ab402fbe2350d5382c972ae7102bd53258a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:18:22 -0600 Subject: [PATCH 06/21] feat: add stash mode to CometExecIterator for batch handle output --- .../org/apache/comet/CometExecIterator.scala | 104 ++++++++++++++---- 1 file changed, 80 insertions(+), 24 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index f0c6373149..a7b45089b6 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -68,7 +68,8 @@ class CometExecIterator( partitionIndex: Int, broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, encryptedFilePaths: Seq[String] = Seq.empty, - shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty) + shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty, + handleInputs: Array[Object] = Array.empty) extends Iterator[ColumnarBatch] with Logging { @@ -79,14 +80,22 @@ class CometExecIterator( private val taskAttemptId = TaskContext.get().taskAttemptId private val taskCPUs = TaskContext.get().cpus() private val cometTaskMemoryManager = new CometTaskMemoryManager(id, taskAttemptId) + // When true, executePlan stashes output batches natively and returns handles + // instead of exporting via Arrow FFI. Used when output feeds a native ShuffleWriter. + @volatile private var stashMode: Boolean = false + private var pendingHandle: Long = -1L // Build a mixed array of iterators: CometShuffleBlockIterator for shuffle // scan indices, CometBatchIterator for regular scan indices. - private val inputIterators: Array[Object] = inputs.zipWithIndex.map { - case (_, idx) if shuffleBlockIterators.contains(idx) => - shuffleBlockIterators(idx).asInstanceOf[Object] - case (iterator, _) => - new CometBatchIterator(iterator, nativeUtil).asInstanceOf[Object] - }.toArray + private val inputIterators: Array[Object] = if (handleInputs.nonEmpty) { + handleInputs + } else { + inputs.zipWithIndex.map { + case (_, idx) if shuffleBlockIterators.contains(idx) => + shuffleBlockIterators(idx).asInstanceOf[Object] + case (iterator, _) => + new CometBatchIterator(iterator, nativeUtil).asInstanceOf[Object] + }.toArray + } private val plan = { val conf = SparkEnv.get.conf @@ -189,31 +198,48 @@ class CometExecIterator( override def hasNext: Boolean = { if (closed) return false - if (nextBatch.isDefined) { - return true - } + if (stashMode) { + if (pendingHandle >= 0) return true + val ctx = TaskContext.get() + pendingHandle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) + if (pendingHandle == -1L) { + close() + false + } else { + true + } + } else { + if (nextBatch.isDefined) { + return true + } - // Close previous batch if any. - // This is to guarantee safety at the native side before we overwrite the buffer memory - // shared across batches in the native side. - if (prevBatch != null) { - prevBatch.close() - prevBatch = null - } + // Close previous batch if any. + // This is to guarantee safety at the native side before we overwrite the buffer memory + // shared across batches in the native side. + if (prevBatch != null) { + prevBatch.close() + prevBatch = null + } - nextBatch = getNextBatch + nextBatch = getNextBatch - logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") + logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") - if (nextBatch.isEmpty) { - close() - false - } else { - true + if (nextBatch.isEmpty) { + close() + false + } else { + true + } } } override def next(): ColumnarBatch = { + if (stashMode) { + throw new UnsupportedOperationException( + "next() should not be called in stash mode. Use nextHandle() instead.") + } + if (currentBatch != null) { // Eagerly release Arrow Arrays in the previous batch currentBatch.close() @@ -230,6 +256,36 @@ class CometExecIterator( currentBatch } + /** Enable stash mode. Must be called before iteration begins. */ + def enableStashMode(): Unit = { + stashMode = true + } + + /** + * In stash mode, advance the native plan and return the batch handle. + * Returns a positive handle, or -1 for EOF. + */ + def nextHandle(): Long = { + if (closed) return -1L + + if (pendingHandle >= 0) { + val h = pendingHandle + pendingHandle = -1L + return h + } + + val ctx = TaskContext.get() + try { + val handle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) + if (handle == -1L) { + close() + } + handle + } catch { + case e: Throwable => throw e + } + } + def close(): Unit = synchronized { if (!closed) { if (currentBatch != null) { From 0a49ca6488ec686ef2f6352c1d4f80d8db7d980c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:19:49 -0600 Subject: [PATCH 07/21] feat: add handle-mode input path to ScanExec for batch stash retrieval --- native/core/src/execution/operators/scan.rs | 61 +++++++++++++++++++-- 1 file changed, 55 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 90bb741b5e..5fa2e111f7 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -77,6 +77,9 @@ pub struct ScanExec { baseline_metrics: BaselineMetrics, /// Whether native code can assume ownership of batches that it receives arrow_ffi_safe: bool, + /// When true, input comes from a CometHandleBatchIterator and batches are + /// retrieved from the BatchStash instead of via Arrow FFI import. + pub handle_mode: bool, } impl ScanExec { @@ -113,6 +116,7 @@ impl ScanExec { baseline_metrics, schema, arrow_ffi_safe, + handle_mode: false, }) } @@ -141,12 +145,19 @@ impl ScanExec { let mut current_batch = self.batch.try_lock().unwrap(); if current_batch.is_none() { - let next_batch = ScanExec::get_next( - self.exec_context_id, - self.input_source.as_ref().unwrap().as_obj(), - self.data_types.len(), - self.arrow_ffi_safe, - )?; + let next_batch = if self.handle_mode { + ScanExec::get_next_handle( + self.exec_context_id, + self.input_source.as_ref().unwrap().as_obj(), + )? + } else { + ScanExec::get_next( + self.exec_context_id, + self.input_source.as_ref().unwrap().as_obj(), + self.data_types.len(), + self.arrow_ffi_safe, + )? + }; *current_batch = Some(next_batch); } @@ -257,6 +268,44 @@ impl ScanExec { }) } + /// Pull next input batch from a CometHandleBatchIterator via batch stash handle. + fn get_next_handle( + exec_context_id: i64, + iter: &JObject, + ) -> Result { + if exec_context_id == TEST_EXEC_CONTEXT_ID { + return Ok(InputBatch::EOF); + } + + if iter.is_null() { + return Err(CometError::from(ExecutionError::GeneralError(format!( + "Null handle batch iterator object. Plan id: {exec_context_id}" + )))); + } + + JVMClasses::with_env(|env| { + let handle: i64 = unsafe { + jni_call!(env, + comet_handle_batch_iterator(iter).next_handle() -> i64)? + }; + + if handle == -1 { + return Ok(InputBatch::EOF); + } + + match crate::execution::batch_stash::take(handle as u64) { + Some(batch) => { + let arrays: Vec = batch.columns().to_vec(); + let num_rows = batch.num_rows(); + Ok(InputBatch::new(arrays, Some(num_rows))) + } + None => Err(CometError::from(ExecutionError::GeneralError(format!( + "Batch stash handle {handle} not found" + )))), + } + }) + } + /// Allocates Arrow FFI structures and calls JNI to get the next batch data. /// Returns the number of rows and the allocated array/schema addresses. fn allocate_and_fetch_batch( From 84862efddde37bdcaea0911e360f336c75950d2b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:21:09 -0600 Subject: [PATCH 08/21] feat: detect CometHandleBatchIterator in planner and enable handle mode --- native/core/src/execution/planner.rs | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index ac35925ace..4c747db38d 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1296,14 +1296,29 @@ impl PhysicalPlanner { }; // The `ScanExec` operator will take actual arrays from Spark during execution - let scan = ScanExec::new( + let mut scan = ScanExec::new( self.exec_context_id, - input_source, + input_source.clone(), &scan.source, data_types, scan.arrow_ffi_safe, )?; + // Check if the input source is a CometHandleBatchIterator. + // If so, enable handle mode on the scan to retrieve batches from + // the BatchStash instead of via Arrow FFI. + if let Some(ref source) = input_source { + let is_handle_iter = crate::jvm_bridge::JVMClasses::with_env(|env| { + let handle_class = + &crate::jvm_bridge::JVMClasses::get().comet_handle_batch_iterator.class; + let result = env.is_instance_of(source.as_obj(), handle_class)?; + Ok::(result) + })?; + if is_handle_iter { + scan.handle_mode = true; + } + } + Ok(( vec![scan.clone()], vec![], From 35885431cd8e267550b48242ad9be082a64c75f2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:22:32 -0600 Subject: [PATCH 09/21] feat: preserve CometExecIterator reference through shuffle dependency RDD --- .../shuffle/CometShuffleExchangeExec.scala | 26 ++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index df2dca0331..94e97e1391 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -50,6 +50,7 @@ import com.google.common.base.Objects import org.apache.comet.CometConf import org.apache.comet.CometConf.{COMET_EXEC_SHUFFLE_ENABLED, COMET_SHUFFLE_MODE} +import org.apache.comet.CometExecIterator import org.apache.comet.CometSparkSessionExtensions.{isCometShuffleManagerEnabled, withInfo} import org.apache.comet.serde.{Compatible, OperatorOuterClass, QueryPlanSerde, SupportLevel, Unsupported} import org.apache.comet.serde.operator.CometSink @@ -640,10 +641,16 @@ object CometShuffleExchangeExec None) } + val wrappedRDD = rdd.mapPartitions { iter => + val nativeIter = iter match { + case cei: CometExecIterator => Some(cei) + case _ => None + } + new CometShuffleWriterInputIterator(iter, nativeIter) + } + val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( - rdd.map( - (0, _) - ), // adding fake partitionId that is always 0 because ShuffleDependency requires it + wrappedRDD, serializer = serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(metrics), shuffleType = CometNativeShuffle, @@ -858,3 +865,16 @@ object CometShuffleExchangeExec dependency } } + +/** + * An iterator wrapper that preserves access to the underlying CometExecIterator + * when present. Used by CometNativeShuffleWriter to detect native child plans + * and enable the batch stash optimization. + */ +private[shuffle] class CometShuffleWriterInputIterator( + underlying: Iterator[ColumnarBatch], + val nativeIterator: Option[CometExecIterator]) + extends Iterator[Product2[Int, ColumnarBatch]] { + override def hasNext: Boolean = underlying.hasNext + override def next(): Product2[Int, ColumnarBatch] = (0, underlying.next()) +} From db137ad6cc73e57e5ae61e3d88d44903dd46d59a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:23:45 -0600 Subject: [PATCH 10/21] feat: integrate batch stash mode in CometNativeShuffleWriter --- .../shuffle/CometNativeShuffleWriter.scala | 45 +++++++++++++------ .../apache/spark/sql/comet/operators.scala | 23 ++++++++++ 2 files changed, 55 insertions(+), 13 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index f27d021ac4..1a2a2260a1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.comet.CometConf +import org.apache.comet.{CometConf, CometExecIterator, CometHandleBatchIterator} import org.apache.comet.serde.{OperatorOuterClass, PartitioningOuterClass, QueryPlanSerde} import org.apache.comet.serde.OperatorOuterClass.{CompressionCodec, Operator} import org.apache.comet.serde.QueryPlanSerde.serializeDataType @@ -93,18 +93,37 @@ class CometNativeShuffleWriter[K, V]( metrics.filterKeys(detailedMetrics.contains) val nativeMetrics = CometMetricNode(nativeSQLMetrics) - // Getting rid of the fake partitionId - val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) - - val cometIter = CometExec.getCometIterator( - Seq(newInputs.asInstanceOf[Iterator[ColumnarBatch]]), - outputAttributes.length, - nativePlan, - nativeMetrics, - numParts, - context.partitionId(), - broadcastedHadoopConfForEncryption = None, - encryptedFilePaths = Seq.empty) + // Detect if input comes from a native plan (CometExecIterator) + val nativeIter: Option[CometExecIterator] = inputs match { + case swi: CometShuffleWriterInputIterator => swi.nativeIterator + case _ => None + } + + val cometIter = nativeIter match { + case Some(childIter) => + // Stash mode: child plan stashes batches, shuffle writer retrieves via handles + childIter.enableStashMode() + val handleIter = new CometHandleBatchIterator(childIter) + CometExec.getCometIteratorWithHandleInputs( + Array(handleIter.asInstanceOf[Object]), + outputAttributes.length, + nativePlan, + nativeMetrics, + numParts, + context.partitionId()) + case None => + // Normal FFI mode: wrap input in CometBatchIterator as before + val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) + CometExec.getCometIterator( + Seq(newInputs.asInstanceOf[Iterator[ColumnarBatch]]), + outputAttributes.length, + nativePlan, + nativeMetrics, + numParts, + context.partitionId(), + broadcastedHadoopConfForEncryption = None, + encryptedFilePaths = Seq.empty) + } while (cometIter.hasNext) { cometIter.next() diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 21cbdab974..dec2e7fbd1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -370,6 +370,29 @@ object CometExec { encryptedFilePaths) } + /** + * Create a CometExecIterator with pre-built input iterators (e.g., CometHandleBatchIterator). + * Bypasses the normal CometBatchIterator wrapping. + */ + def getCometIteratorWithHandleInputs( + handleInputs: Array[Object], + numOutputCols: Int, + nativePlan: Operator, + nativeMetrics: CometMetricNode, + numParts: Int, + partitionIdx: Int): CometExecIterator = { + val bytes = serializeNativePlan(nativePlan) + new CometExecIterator( + newIterId, + Seq.empty, + numOutputCols, + bytes, + nativeMetrics, + numParts, + partitionIdx, + handleInputs = handleInputs) + } + /** * Executes this Comet operator and serialized output ColumnarBatch into bytes. */ From ed95e832026c9778aceabce059bf97aa898f1c62 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 08:54:57 -0600 Subject: [PATCH 11/21] style: apply formatting fixes --- native/Cargo.lock | 4 +--- .../org/apache/comet/CometHandleBatchIterator.java | 4 ++-- .../scala/org/apache/comet/CometExecIterator.scala | 7 ++++--- spark/src/main/scala/org/apache/comet/Native.scala | 11 ++++------- .../execution/shuffle/CometShuffleExchangeExec.scala | 6 +++--- 5 files changed, 14 insertions(+), 18 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 480f7ad06d..b5c7f2b0c7 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -2070,9 +2070,7 @@ dependencies = [ "num", "rand 0.10.0", "regex", - "serde", "serde_json", - "thiserror 2.0.18", "tokio", "twox-hash", ] @@ -5885,7 +5883,7 @@ version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "83fc039473c5595ace860d8c4fafa220ff474b3fc6bfdb4293327f1a37e94d86" dependencies = [ - "indexmap 2.13.0", + "indexmap 2.13.1", "itoa", "memchr", "serde", diff --git a/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java b/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java index dbc43f7d71..98cef1d5e8 100644 --- a/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java +++ b/spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java @@ -21,8 +21,8 @@ /** * Iterator that passes opaque native batch handles between two native execution contexts through - * the JVM. Used when a native child plan feeds directly into a native ShuffleWriter, avoiding - * Arrow FFI export/import overhead. + * the JVM. Used when a native child plan feeds directly into a native ShuffleWriter, avoiding Arrow + * FFI export/import overhead. * *

Called from native ScanExec via JNI. The source CometExecIterator must be in stash mode. */ diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index a7b45089b6..d939504a33 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -223,7 +223,8 @@ class CometExecIterator( nextBatch = getNextBatch - logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") + logTrace( + s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") if (nextBatch.isEmpty) { close() @@ -262,8 +263,8 @@ class CometExecIterator( } /** - * In stash mode, advance the native plan and return the batch handle. - * Returns a positive handle, or -1 for EOF. + * In stash mode, advance the native plan and return the batch handle. Returns a positive + * handle, or -1 for EOF. */ def nextHandle(): Long = { if (closed) return -1L diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 55d0555970..ff709d2b23 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -97,9 +97,9 @@ class Native extends NativeBase { /** * Execute one step of the native query plan, stashing the output RecordBatch in the native - * BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) - * or -1 for EOF. Used when the output feeds directly into another native plan (e.g., - * native ShuffleWriter) to avoid unnecessary FFI round-trips. + * BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) or -1 + * for EOF. Used when the output feeds directly into another native plan (e.g., native + * ShuffleWriter) to avoid unnecessary FFI round-trips. * * @param stage * the stage ID, for informational purposes @@ -110,10 +110,7 @@ class Native extends NativeBase { * @return * a batch stash handle (positive), or -1 for EOF. */ - @native def executePlanBatchHandle( - stage: Int, - partition: Int, - plan: Long): Long + @native def executePlanBatchHandle(stage: Int, partition: Int, plan: Long): Long /** * Release and drop the native query plan object and context object. diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 94e97e1391..40b231e090 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -867,9 +867,9 @@ object CometShuffleExchangeExec } /** - * An iterator wrapper that preserves access to the underlying CometExecIterator - * when present. Used by CometNativeShuffleWriter to detect native child plans - * and enable the batch stash optimization. + * An iterator wrapper that preserves access to the underlying CometExecIterator when present. + * Used by CometNativeShuffleWriter to detect native child plans and enable the batch stash + * optimization. */ private[shuffle] class CometShuffleWriterInputIterator( underlying: Iterator[ColumnarBatch], From a827437bf24177339b320c6a213a0605d6db9e36 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:15:13 -0600 Subject: [PATCH 12/21] fix: use scan source name for handle mode detection instead of JNI is_instance_of --- native/core/src/execution/batch_stash.rs | 3 +- native/core/src/execution/operators/scan.rs | 5 +- native/core/src/execution/planner.rs | 19 +- spark/hs_err_pid58786.log | 1611 +++++++ spark/hs_err_pid58919.log | 1602 +++++++ spark/hs_err_pid59058.log | 1568 +++++++ spark/hs_err_pid59157.log | 3785 +++++++++++++++++ spark/hs_err_pid59323.log | 1626 +++++++ spark/hs_err_pid59395.log | 1886 ++++++++ .../shuffle/CometNativeShuffleWriter.scala | 23 +- 10 files changed, 12100 insertions(+), 28 deletions(-) create mode 100644 spark/hs_err_pid58786.log create mode 100644 spark/hs_err_pid58919.log create mode 100644 spark/hs_err_pid59058.log create mode 100644 spark/hs_err_pid59157.log create mode 100644 spark/hs_err_pid59323.log create mode 100644 spark/hs_err_pid59395.log diff --git a/native/core/src/execution/batch_stash.rs b/native/core/src/execution/batch_stash.rs index a5eaad42cc..d34f54f885 100644 --- a/native/core/src/execution/batch_stash.rs +++ b/native/core/src/execution/batch_stash.rs @@ -31,8 +31,7 @@ use std::sync::Mutex; static NEXT_HANDLE: AtomicU64 = AtomicU64::new(1); /// Global stash mapping handles to RecordBatch values. -static STASH: Lazy>> = - Lazy::new(|| Mutex::new(HashMap::new())); +static STASH: Lazy>> = Lazy::new(|| Mutex::new(HashMap::new())); /// Store a RecordBatch in the global stash and return a unique handle. pub(crate) fn stash(batch: RecordBatch) -> u64 { diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 5fa2e111f7..e1770ce519 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -269,10 +269,7 @@ impl ScanExec { } /// Pull next input batch from a CometHandleBatchIterator via batch stash handle. - fn get_next_handle( - exec_context_id: i64, - iter: &JObject, - ) -> Result { + fn get_next_handle(exec_context_id: i64, iter: &JObject) -> Result { if exec_context_id == TEST_EXEC_CONTEXT_ID { return Ok(InputBatch::EOF); } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 4c747db38d..4e0b6f58b9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1298,25 +1298,18 @@ impl PhysicalPlanner { // The `ScanExec` operator will take actual arrays from Spark during execution let mut scan = ScanExec::new( self.exec_context_id, - input_source.clone(), + input_source, &scan.source, data_types, scan.arrow_ffi_safe, )?; - // Check if the input source is a CometHandleBatchIterator. - // If so, enable handle mode on the scan to retrieve batches from + // Check if the scan source indicates handle mode (batch stash path). + // When the JVM sets source to "ShuffleWriterInputHandle", the input + // is a CometHandleBatchIterator and batches should be retrieved from // the BatchStash instead of via Arrow FFI. - if let Some(ref source) = input_source { - let is_handle_iter = crate::jvm_bridge::JVMClasses::with_env(|env| { - let handle_class = - &crate::jvm_bridge::JVMClasses::get().comet_handle_batch_iterator.class; - let result = env.is_instance_of(source.as_obj(), handle_class)?; - Ok::(result) - })?; - if is_handle_iter { - scan.handle_mode = true; - } + if scan.input_source_description == "ShuffleWriterInputHandle" { + scan.handle_mode = true; } Ok(( diff --git a/spark/hs_err_pid58786.log b/spark/hs_err_pid58786.log new file mode 100644 index 0000000000..26eadbb201 --- /dev/null +++ b/spark/hs_err_pid58786.log @@ -0,0 +1,1611 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x0000000106d89f64, pid=58786, tid=67331 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 08:56:28 2026 MDT elapsed time: 21.994476 seconds (0d 0h 0m 21s) + +--------------- T H R E A D --------------- + +Current thread (0x0000000755abea00): JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=67331, stack(0x0000000401834000,0x0000000401c37000)] + +Stack: [0x0000000401834000,0x0000000401c37000], sp=0x0000000401c1e120, free space=4008k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +C [libcomet-14714718128898053499.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 +C [libcomet-14714718128898053499.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 +C [libcomet-14714718128898053499.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c +C [libcomet-14714718128898053499.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 +C [libcomet-14714718128898053499.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 +C [libcomet-14714718128898053499.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 +C [libcomet-14714718128898053499.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 +C [libcomet-14714718128898053499.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-14714718128898053499.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-14714718128898053499.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-14714718128898053499.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-14714718128898053499.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-14714718128898053499.dylib+0x106628] __rust_try+0x20 +C [libcomet-14714718128898053499.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-14714718128898053499.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-14714718128898053499.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-14714718128898053499.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3960+0x000000f8021c6e40.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3955+0x000000f8021c5a70.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2521+0x000000f801df1e28.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3960+0x000000f8021c6e40.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3955+0x000000f8021c5a70.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2521+0x000000f801df1e28.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x656c666675685343 + +Registers: + x0=0x000000f8021af168 x1=0x0000000000000010 x2=0x000000075c4e85e0 x3=0x0000000401c23298 + x4=0x0000000000000238 x5=0x0000000000000018 x6=0x000000075c77c8a0 x7=0xfffff0003ffff800 + x8=0x00000000021af168 x9=0x0000000000000000 x10=0x000000f800000000 x11=0x0000000753584040 +x12=0x000000075c06abc0 x13=0x0000000000000001 x14=0x0000000106479a68 x15=0x0000000753584000 +x16=0x000000018c8009d0 x17=0x00000007529a0500 x18=0x0000000000000000 x19=0x0000000755abea00 +x20=0x0000000000000001 x21=0x00000001014ee248 x22=0x656c66667568532f x23=0x00000001074e9088 +x24=0x0000000000000000 x25=0x00000001074edf38 x26=0x0000000134fda0a8 x27=0x00000003033a7dc0 +x28=0x0000000755abea00 fp=0x0000000401c1e160 lr=0x0000000106d89f38 sp=0x0000000401c1e120 +pc=0x0000000106d89f64 cpsr=0x0000000040001000 + +Register to memory mapping: + + x0=0x000000f8021af168 is a pointer to class: +org.apache.comet.CometHandleBatchIterator {0x000000f8021af168} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x0000000135051340) + - method ordering: Array(0x000000f800913f08) + - local interfaces: Array(0x000000f8005d4fb8) + - trans. interfaces: Array(0x000000f8005d4fb8) + - constants: constant pool [62]/operands[4] {0x00000001350510b0} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000135051590 + - class loader data: loader data: 0x000000075c020fa0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x000000030330ac58} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x000000f8005d4f48) + - nest members: Array(0x000000f8005d4f48) + - permitted subclasses: Array(0x000000f8005d4f48) + - java mirror: a 'java/lang/Class'{0x0000000313c484f8} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x000000f8021af338) + - itable length 2 (start addr: 0x000000f8021af368) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x1=0x0000000000000010 is an unknown value + x2=0x000000075c4e85e0 points into unknown readable memory: 0x000000032a664cf0 | f0 4c 66 2a 03 00 00 00 + x3=0x0000000401c23298 is pointing into the stack for thread: 0x0000000755abea00 + x4=0x0000000000000238 is an unknown value + x5=0x0000000000000018 is an unknown value + x6=0x000000075c77c8a0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=35320168 is a compressed pointer to class: 0x000000f8021af168 +org.apache.comet.CometHandleBatchIterator {0x000000f8021af168} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x0000000135051340) + - method ordering: Array(0x000000f800913f08) + - local interfaces: Array(0x000000f8005d4fb8) + - trans. interfaces: Array(0x000000f8005d4fb8) + - constants: constant pool [62]/operands[4] {0x00000001350510b0} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000135051590 + - class loader data: loader data: 0x000000075c020fa0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x000000030330ac58} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x000000f8005d4f48) + - nest members: Array(0x000000f8005d4f48) + - permitted subclasses: Array(0x000000f8005d4f48) + - java mirror: a 'java/lang/Class'{0x0000000313c484f8} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x000000f8021af338) + - itable length 2 (start addr: 0x000000f8021af368) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x9=0x0 is NULL +x10=0x000000f800000000 is pointing into metadata +x11=0x0000000753584040 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +x12=0x000000075c06abc0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x13=0x0000000000000001 is an unknown value +x14=0x0000000106479a68 points into unknown readable memory: 0x02d626b000c7e800 | 00 e8 c7 00 b0 26 d6 02 +x15=0x0000000753584000 points into unknown readable memory: 0x4a191953546ffd27 | 27 fd 6f 54 53 19 19 4a +x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 +x17=0x00000007529a0500 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +x18=0x0 is NULL +x19=0x0000000755abea00 is a thread +x20=0x0000000000000001 is an unknown value +x21=0x00000001014ee248 is a global jni handle +x22=0x656c66667568532f is an unknown value +x23=0x00000001074e9088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 +x24=0x0 is NULL +x25=0x00000001074edf38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 +x26=0x0000000134fda0a8 is pointing into metadata +x27=0x00000003033a7dc0 is an oop: [Z +{0x00000003033a7dc0} - klass: {type array bool} + - length: 9 +x28=0x0000000755abea00 is a thread + + +Top of Stack: (sp=0x0000000401c1e120) +0x0000000401c1e120: 0000000134fda0a8 000000030331a520 +0x0000000401c1e130: 0000000401c35818 0000000303319f00 +0x0000000401c1e140: 0000000000000000 0000000107537f90 +0x0000000401c1e150: 0000000401c35738 000000011687372c +0x0000000401c1e160: 0000000401c1e5f0 000000014c74ce4c +0x0000000401c1e170: 0000000000000000 000000014f66e880 +0x0000000401c1e180: 0000000401c1e190 000000014c786d54 +0x0000000401c1e190: 000000075c4e85e0 00000001014ee248 +0x0000000401c1e1a0: 0000000106d89e08 00000007586547b0 +0x0000000401c1e1b0: 00000001074f2ad0 00000001074f2ad0 +0x0000000401c1e1c0: 0000000755abecb0 0000000755abecb0 +0x0000000401c1e1d0: 000000014f67ae90 000000004c74ee14 +0x0000000401c1e1e0: 0000000000000000 0000000401c1e270 +0x0000000401c1e1f0: 0000000755abecb0 000000014f67ae90 +0x0000000401c1e200: 0000000055abea00 0000000401c1e860 +0x0000000401c1e210: 0000000401c1ea00 8000000000000020 +0x0000000401c1e220: 000000014f67ae90 00000007586547b0 +0x0000000401c1e230: 000000014f67ae90 8000000000000020 +0x0000000401c1e240: 000000014f67ae90 0000000755abecb0 +0x0000000401c1e250: 0000000755abecb0 0000000000000002 +0x0000000401c1e260: 00000007529a0500 0000000000000000 +0x0000000401c1e270: 0000000755abecb0 0000000000000002 +0x0000000401c1e280: 00000007529a0500 8000000000000020 +0x0000000401c1e290: 000000014f67ae90 00000007529a05d0 +0x0000000401c1e2a0: 0000000401c1e2e0 000000014c7afc28 +0x0000000401c1e2b0: 0000000000000000 0000000000000000 +0x0000000401c1e2c0: 0000000755abecb0 000000014f66e888 +0x0000000401c1e2d0: 0000000000000000 0000000755abecb0 +0x0000000401c1e2e0: 0000000755abecb0 0000000000000002 +0x0000000401c1e2f0: 0000000000000008 000000014f66e888 +0x0000000401c1e300: 0000000401c1e340 000000014c787d8c +0x0000000401c1e310: 0000000401c1e3f8 0000000755abecb0 + +Instructions: (pc=0x0000000106d89f64) +0x0000000106d89e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 +0x0000000106d89e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 +0x0000000106d89e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 +0x0000000106d89e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 +0x0000000106d89ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 +0x0000000106d89eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 +0x0000000106d89ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 +0x0000000106d89ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 +0x0000000106d89ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 +0x0000000106d89ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a +0x0000000106d89f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 +0x0000000106d89f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa +0x0000000106d89f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 +0x0000000106d89f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 +0x0000000106d89f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 +0x0000000106d89f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b +0x0000000106d89f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 +0x0000000106d89f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 +0x0000000106d89f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 +0x0000000106d89f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb +0x0000000106d89fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 +0x0000000106d89fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 +0x0000000106d89fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 +0x0000000106d89fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 +0x0000000106d89fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 +0x0000000106d89ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 +0x0000000106d8a004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b +0x0000000106d8a014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 +0x0000000106d8a024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 +0x0000000106d8a034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 +0x0000000106d8a044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 +0x0000000106d8a054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x0000000134fda0a8 is pointing into metadata +stack at sp + 1 slots: 0x000000030331a520 is an oop: [Z +{0x000000030331a520} - klass: {type array bool} + - length: 264 +stack at sp + 2 slots: 0x0000000401c35818 is pointing into the stack for thread: 0x0000000755abea00 +stack at sp + 3 slots: 0x0000000303319f00 is an oop: [Z +{0x0000000303319f00} - klass: {type array bool} + - length: 187 +stack at sp + 4 slots: 0x0 is NULL +stack at sp + 5 slots: 0x0000000107537f90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 +stack at sp + 6 slots: 0x0000000401c35738 is pointing into the stack for thread: 0x0000000755abea00 +stack at sp + 7 slots: 0x000000011687372c is at code_begin+44 in an Interpreter codelet +result handlers for native calls [0x0000000116873700, 0x0000000116873780] 128 bytes + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x0000000764145140, length=77, elements={ +0x00000001014d1e20, 0x00000001014e6da0, 0x0000000758d80000, 0x0000000758d80c00, +0x0000000758d81200, 0x0000000758d81800, 0x0000000758d81e00, 0x0000000758d82400, +0x0000000758d82a00, 0x0000000758d83000, 0x0000000758e9e400, 0x0000000755245800, +0x000000075589d200, 0x000000075589e400, 0x000000075589ea00, 0x000000075589f000, +0x000000075589f600, 0x000000075586c600, 0x00000007552a7600, 0x0000000756ff0000, +0x0000000755246400, 0x0000000755244c00, 0x0000000758350000, 0x0000000758350600, +0x0000000758350c00, 0x0000000758351200, 0x0000000758353600, 0x0000000758375800, +0x0000000758375e00, 0x0000000758377000, 0x0000000758377600, 0x000000075839c600, +0x000000075839cc00, 0x000000075839d200, 0x000000075839de00, 0x000000075839e400, +0x000000075839f600, 0x00000007583c0000, 0x00000007583c0c00, 0x00000007583c1800, +0x00000007583c1e00, 0x00000007583c2400, 0x00000007583c2a00, 0x0000000754843000, +0x000000075488cc00, 0x000000075488d800, 0x0000000754970600, 0x0000000754670000, +0x0000000755abd200, 0x0000000755abd800, 0x0000000755abde00, 0x0000000755abe400, +0x0000000755abea00, 0x0000000755acaa00, 0x0000000755abf600, 0x0000000754789e00, +0x00000007575af600, 0x0000000754d86a00, 0x00000007544cb000, 0x0000000755ac8000, +0x0000000755ac8600, 0x0000000752906a00, 0x0000000752907000, 0x0000000752907600, +0x0000000752974000, 0x0000000752974600, 0x0000000752b5d200, 0x0000000752b5d800, +0x0000000752b5de00, 0x0000000752b5e400, 0x0000000752b5ea00, 0x0000000758e9f600, +0x0000000755abf000, 0x0000000755abcc00, 0x0000000755acb000, 0x0000000756ff3000, +0x00000007535b4000 +} + +Java Threads: ( => current thread ) + 0x00000001014d1e20 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5891, stack(0x000000016f0f0000,0x000000016f4f3000)] + 0x00000001014e6da0 JavaThread "Reference Handler" daemon [_thread_blocked, id=22787, stack(0x0000000170144000,0x0000000170547000)] + 0x0000000758d80000 JavaThread "Finalizer" daemon [_thread_blocked, id=30723, stack(0x0000000170550000,0x0000000170953000)] + 0x0000000758d80c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=29955, stack(0x0000000170a74000,0x0000000170e77000)] + 0x0000000758d81200 JavaThread "Service Thread" daemon [_thread_blocked, id=29443, stack(0x0000000170e80000,0x0000000171283000)] + 0x0000000758d81800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=25859, stack(0x000000017128c000,0x000000017168f000)] + 0x0000000758d81e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26371, stack(0x0000000171698000,0x000000017189b000)] + 0x0000000758d82400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=29187, stack(0x00000001718a4000,0x0000000171aa7000)] + 0x0000000758d82a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=28675, stack(0x0000000171ab0000,0x0000000171eb3000)] + 0x0000000758d83000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=26883, stack(0x0000000171ebc000,0x00000001722bf000)] + 0x0000000758e9e400 JavaThread "Notification Thread" daemon [_thread_blocked, id=33539, stack(0x0000000172d04000,0x0000000173107000)] + 0x0000000755245800 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45315, stack(0x0000000176024000,0x0000000176427000)] + 0x000000075589d200 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=62723, stack(0x0000000176e6c000,0x000000017726f000)] + 0x000000075589e400 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=62467, stack(0x0000000177278000,0x000000017767b000)] + 0x000000075589ea00 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=47619, stack(0x0000000177684000,0x0000000177a87000)] + 0x000000075589f000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=47875, stack(0x0000000177a90000,0x0000000177e93000)] + 0x000000075589f600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=61955, stack(0x0000000177e9c000,0x000000017829f000)] + 0x000000075586c600 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=48387, stack(0x00000001782a8000,0x00000001786ab000)] + 0x00000007552a7600 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=48899, stack(0x00000001786b4000,0x0000000178ab7000)] + 0x0000000756ff0000 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=61187, stack(0x0000000178ac0000,0x0000000178ec3000)] + 0x0000000755246400 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=60675, stack(0x0000000178ecc000,0x00000001792cf000)] + 0x0000000755244c00 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=60419, stack(0x00000001792d8000,0x00000001796db000)] + 0x0000000758350000 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=59907, stack(0x00000001796e4000,0x0000000179ae7000)] + 0x0000000758350600 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=59651, stack(0x0000000179af0000,0x0000000179ef3000)] + 0x0000000758350c00 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49923, stack(0x0000000179efc000,0x000000017a2ff000)] + 0x0000000758351200 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=50435, stack(0x000000017a308000,0x000000017a70b000)] + 0x0000000758353600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50691, stack(0x000000017a714000,0x000000017ab17000)] + 0x0000000758375800 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=59139, stack(0x000000017ab20000,0x000000017af23000)] + 0x0000000758375e00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=58883, stack(0x000000017af2c000,0x000000017b32f000)] + 0x0000000758377000 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58371, stack(0x000000017b338000,0x000000017b73b000)] + 0x0000000758377600 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=58115, stack(0x000000017b744000,0x000000017bb47000)] + 0x000000075839c600 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=57859, stack(0x000000017bb50000,0x000000017bf53000)] + 0x000000075839cc00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57347, stack(0x000000017bf5c000,0x000000017c35f000)] + 0x000000075839d200 JavaThread "task-abort-timer" daemon [_thread_blocked, id=57091, stack(0x000000017c368000,0x000000017c76b000)] + 0x000000075839de00 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=56579, stack(0x000000017c774000,0x000000017cb77000)] + 0x000000075839e400 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52483, stack(0x000000017cb80000,0x000000017cf83000)] + 0x000000075839f600 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=52739, stack(0x000000017cf8c000,0x000000017d38f000)] + 0x00000007583c0000 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=56067, stack(0x000000017d398000,0x000000017d79b000)] + 0x00000007583c0c00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=53507, stack(0x000000017d7a4000,0x000000017dba7000)] + 0x00000007583c1800 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=55811, stack(0x000000017dbb0000,0x000000017dfb3000)] + 0x00000007583c1e00 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54275, stack(0x000000017dfbc000,0x000000017e3bf000)] + 0x00000007583c2400 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=55555, stack(0x000000017e3c8000,0x000000017e7cb000)] + 0x00000007583c2a00 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55299, stack(0x000000017e7d4000,0x000000017ebd7000)] + 0x0000000754843000 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=65539, stack(0x000000017ebe0000,0x000000017efe3000)] + 0x000000075488cc00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=87043, stack(0x000000017efec000,0x000000017f3ef000)] + 0x000000075488d800 JavaThread "Timer-0" [_thread_blocked, id=86531, stack(0x000000017f3f8000,0x000000017f7fb000)] + 0x0000000754970600 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=66051, stack(0x000000017f804000,0x000000017fc07000)] + 0x0000000754670000 JavaThread "process reaper" daemon [_thread_blocked, id=86027, stack(0x000000017fc10000,0x000000017fc47000)] + 0x0000000755abd200 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x0000000400804000,0x0000000400c07000)] + 0x0000000755abd800 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=41523, stack(0x0000000400c10000,0x0000000401013000)] + 0x0000000755abde00 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=30223, stack(0x000000040101c000,0x000000040141f000)] + 0x0000000755abe400 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=85763, stack(0x0000000401428000,0x000000040182b000)] +=>0x0000000755abea00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=67331, stack(0x0000000401834000,0x0000000401c37000)] + 0x0000000755acaa00 JavaThread "process reaper" daemon [_thread_blocked, id=68099, stack(0x00000001709e8000,0x0000000170a1f000)] + 0x0000000755abf600 JavaThread "process reaper" daemon [_thread_blocked, id=83715, stack(0x0000000170a28000,0x0000000170a5f000)] + 0x0000000754789e00 JavaThread "process reaper" daemon [_thread_blocked, id=83459, stack(0x000000017fe5c000,0x000000017fe93000)] + 0x00000007575af600 JavaThread "process reaper" daemon [_thread_blocked, id=69379, stack(0x000000017fe9c000,0x000000017fed3000)] + 0x0000000754d86a00 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=42771, stack(0x000000017331c000,0x000000017371f000)] + 0x00000007544cb000 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=42051, stack(0x0000000401c40000,0x0000000402043000)] + 0x0000000755ac8000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=45839, stack(0x000000040204c000,0x000000040244f000)] + 0x0000000755ac8600 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=67855, stack(0x0000000402458000,0x000000040285b000)] + 0x0000000752906a00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=83267, stack(0x0000000402864000,0x0000000402c67000)] + 0x0000000752907000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=83991, stack(0x0000000402c70000,0x0000000403073000)] + 0x0000000752907600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=70171, stack(0x000000040307c000,0x000000040347f000)] + 0x0000000752974000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=82459, stack(0x0000000403488000,0x000000040388b000)] + 0x0000000752974600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=82179, stack(0x0000000403894000,0x0000000403c97000)] + 0x0000000752b5d200 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=28179, stack(0x00000001722c8000,0x00000001724cb000)] + 0x0000000752b5d800 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=79887, stack(0x00000001724d4000,0x00000001726d7000)] + 0x0000000752b5de00 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=80403, stack(0x00000001726e0000,0x00000001728e3000)] + 0x0000000752b5e400 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=80139, stack(0x00000001728ec000,0x0000000172aef000)] + 0x0000000752b5ea00 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=33331, stack(0x0000000172af8000,0x0000000172cfb000)] + 0x0000000758e9f600 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=43071, stack(0x00000004044d0000,0x00000004048d3000)] + 0x0000000755abf000 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=43327, stack(0x00000004048dc000,0x0000000404cdf000)] + 0x0000000755abcc00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=84771, stack(0x0000000404d00000,0x0000000405103000)] + 0x0000000755acb000 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=84499, stack(0x000000040510c000,0x000000040550f000)] + 0x0000000756ff3000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=83011, stack(0x0000000405518000,0x000000040591b000)] + 0x00000007535b4000 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=72707, stack(0x0000000405924000,0x0000000405d27000)] + +Other Threads: + 0x00000001014e6120 VMThread "VM Thread" [stack: 0x000000016ff38000,0x000000017013b000] [id=20227] + 0x0000000758604600 WatcherThread [stack: 0x0000000173110000,0x0000000173313000] [id=33795] + 0x00000001014de240 GCTaskThread "GC Thread#0" [stack: 0x000000016f4fc000,0x000000016f6ff000] [id=14339] + 0x0000000756f5d200 GCTaskThread "GC Thread#1" [stack: 0x0000000173940000,0x0000000173b43000] [id=41219] + 0x0000000756f5d500 GCTaskThread "GC Thread#2" [stack: 0x0000000173b4c000,0x0000000173d4f000] [id=34819] + 0x0000000756f5d800 GCTaskThread "GC Thread#3" [stack: 0x0000000173d58000,0x0000000173f5b000] [id=35075] + 0x0000000756f5db00 GCTaskThread "GC Thread#4" [stack: 0x0000000173f64000,0x0000000174167000] [id=35587] + 0x0000000756f5de00 GCTaskThread "GC Thread#5" [stack: 0x0000000174170000,0x0000000174373000] [id=35843] + 0x0000000756f5e100 GCTaskThread "GC Thread#6" [stack: 0x000000017437c000,0x000000017457f000] [id=39939] + 0x0000000756f5e400 GCTaskThread "GC Thread#7" [stack: 0x0000000174588000,0x000000017478b000] [id=36099] + 0x0000000756f5e700 GCTaskThread "GC Thread#8" [stack: 0x0000000174794000,0x0000000174997000] [id=36611] + 0x0000000756f5ea00 GCTaskThread "GC Thread#9" [stack: 0x00000001749a0000,0x0000000174ba3000] [id=36867] + 0x0000000756f5ed00 GCTaskThread "GC Thread#10" [stack: 0x0000000174bac000,0x0000000174daf000] [id=38915] + 0x0000000756f5f000 GCTaskThread "GC Thread#11" [stack: 0x0000000174db8000,0x0000000174fbb000] [id=38659] + 0x0000000756f5f300 GCTaskThread "GC Thread#12" [stack: 0x0000000174fc4000,0x00000001751c7000] [id=38403] + 0x0000000756f5f600 GCTaskThread "GC Thread#13" [stack: 0x00000001751d0000,0x00000001753d3000] [id=37635] + 0x0000000756f5f900 GCTaskThread "GC Thread#14" [stack: 0x00000001753dc000,0x00000001755df000] [id=43523] + 0x0000000756f5fc00 GCTaskThread "GC Thread#15" [stack: 0x00000001755e8000,0x00000001757eb000] [id=65283] + 0x00000007567fc000 GCTaskThread "GC Thread#16" [stack: 0x00000001757f4000,0x00000001759f7000] [id=65027] + 0x00000007567fc300 GCTaskThread "GC Thread#17" [stack: 0x0000000175a00000,0x0000000175c03000] [id=64771] + 0x00000007567fc600 GCTaskThread "GC Thread#18" [stack: 0x0000000175c0c000,0x0000000175e0f000] [id=44547] + 0x00000007567fc900 GCTaskThread "GC Thread#19" [stack: 0x0000000175e18000,0x000000017601b000] [id=44803] + 0x00000001014def60 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016f708000,0x000000016f90b000] [id=12547] + 0x00000001014df860 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016f914000,0x000000016fb17000] [id=13059] + 0x000000075534d800 ConcurrentGCThread "G1 Conc#1" [stack: 0x000000017663c000,0x000000017683f000] [id=63747] + 0x000000075534de00 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000176848000,0x0000000176a4b000] [id=63235] + 0x000000075534e100 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000176a54000,0x0000000176c57000] [id=46083] + 0x000000075534e400 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000176c60000,0x0000000176e63000] [id=46595] + 0x00000001014e1b20 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016fb20000,0x000000016fd23000] [id=16643] + 0x00000001014e2420 ConcurrentGCThread "G1 Service" [stack: 0x000000016fd2c000,0x000000016ff2f000] [id=21507] + +Threads with active compile tasks: + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x0000000300800000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x000000f800000000-0x000000f800bc0000-0x000000f800bc0000), size 12320768, SharedBaseAddress: 0x000000f800000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x000000f801000000-0x000000f841000000, reserved size: 1073741824 +Narrow klass base: 0x000000f800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 348160K, used 181790K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 15 young (30720K), 4 survivors (8192K) + Metaspace used 151515K, committed 152832K, reserved 1245184K + class space used 18221K, committed 18816K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000300800000, 0x0000000300a00000, 0x0000000300a00000|100%|HS| |TAMS 0x0000000300a00000, 0x0000000300a00000| Complete +| 1|0x0000000300a00000, 0x0000000300c00000, 0x0000000300c00000|100%|HS| |TAMS 0x0000000300c00000, 0x0000000300c00000| Complete +| 2|0x0000000300c00000, 0x0000000300e00000, 0x0000000300e00000|100%|HC| |TAMS 0x0000000300e00000, 0x0000000300e00000| Complete +| 3|0x0000000300e00000, 0x0000000301000000, 0x0000000301000000|100%|HS| |TAMS 0x0000000301000000, 0x0000000301000000| Complete +| 4|0x0000000301000000, 0x0000000301200000, 0x0000000301200000|100%|HC| |TAMS 0x0000000301200000, 0x0000000301200000| Complete +| 5|0x0000000301200000, 0x0000000301400000, 0x0000000301400000|100%|HC| |TAMS 0x0000000301400000, 0x0000000301400000| Complete +| 6|0x0000000301400000, 0x0000000301600000, 0x0000000301600000|100%|HC| |TAMS 0x0000000301600000, 0x0000000301600000| Complete +| 7|0x0000000301600000, 0x0000000301800000, 0x0000000301800000|100%|HC| |TAMS 0x0000000301800000, 0x0000000301800000| Complete +| 8|0x0000000301800000, 0x0000000301a00000, 0x0000000301a00000|100%|HC| |TAMS 0x0000000301a00000, 0x0000000301a00000| Complete +| 9|0x0000000301a00000, 0x0000000301c00000, 0x0000000301c00000|100%|HC| |TAMS 0x0000000301c00000, 0x0000000301c00000| Complete +| 10|0x0000000301c00000, 0x0000000301e00000, 0x0000000301e00000|100%|HC| |TAMS 0x0000000301e00000, 0x0000000301e00000| Complete +| 11|0x0000000301e00000, 0x0000000302000000, 0x0000000302000000|100%|HS| |TAMS 0x0000000302000000, 0x0000000302000000| Complete +| 12|0x0000000302000000, 0x0000000302200000, 0x0000000302200000|100%|HS| |TAMS 0x0000000302200000, 0x0000000302200000| Complete +| 13|0x0000000302200000, 0x0000000302400000, 0x0000000302400000|100%|HC| |TAMS 0x0000000302400000, 0x0000000302400000| Complete +| 14|0x0000000302400000, 0x0000000302600000, 0x0000000302600000|100%|HS| |TAMS 0x0000000302600000, 0x0000000302600000| Complete +| 15|0x0000000302600000, 0x0000000302800000, 0x0000000302800000|100%|HC| |TAMS 0x0000000302800000, 0x0000000302800000| Complete +| 16|0x0000000302800000, 0x0000000302a00000, 0x0000000302a00000|100%|HS| |TAMS 0x0000000302a00000, 0x0000000302a00000| Complete +| 17|0x0000000302a00000, 0x0000000302c00000, 0x0000000302c00000|100%|HS| |TAMS 0x0000000302c00000, 0x0000000302c00000| Complete +| 18|0x0000000302c00000, 0x0000000302e00000, 0x0000000302e00000|100%|HC| |TAMS 0x0000000302e00000, 0x0000000302e00000| Complete +| 19|0x0000000302e00000, 0x0000000303000000, 0x0000000303000000|100%| O| |TAMS 0x0000000303000000, 0x0000000303000000| Untracked +| 20|0x0000000303000000, 0x0000000303200000, 0x0000000303200000|100%| O| |TAMS 0x0000000303200000, 0x0000000303200000| Untracked +| 21|0x0000000303200000, 0x0000000303400000, 0x0000000303400000|100%| O| |TAMS 0x0000000303400000, 0x0000000303400000| Untracked +| 22|0x0000000303400000, 0x0000000303600000, 0x0000000303600000|100%| O| |TAMS 0x0000000303600000, 0x0000000303600000| Untracked +| 23|0x0000000303600000, 0x0000000303800000, 0x0000000303800000|100%| O| |TAMS 0x0000000303800000, 0x0000000303800000| Untracked +| 24|0x0000000303800000, 0x0000000303a00000, 0x0000000303a00000|100%| O| |TAMS 0x0000000303a00000, 0x0000000303a00000| Untracked +| 25|0x0000000303a00000, 0x0000000303c00000, 0x0000000303c00000|100%| O| |TAMS 0x0000000303c00000, 0x0000000303c00000| Untracked +| 26|0x0000000303c00000, 0x0000000303d9a600, 0x0000000303e00000| 80%| O| |TAMS 0x0000000303d9a600, 0x0000000303d9a600| Untracked +| 27|0x0000000303e00000, 0x0000000304000000, 0x0000000304000000|100%| O| |TAMS 0x0000000304000000, 0x0000000304000000| Untracked +| 28|0x0000000304000000, 0x0000000304163800, 0x0000000304200000| 69%| O| |TAMS 0x0000000304163800, 0x0000000304163800| Untracked +| 29|0x0000000304200000, 0x0000000304400000, 0x0000000304400000|100%| O| |TAMS 0x0000000304400000, 0x0000000304400000| Untracked +| 30|0x0000000304400000, 0x0000000304600000, 0x0000000304600000|100%| O| |TAMS 0x0000000304600000, 0x0000000304600000| Untracked +| 31|0x0000000304600000, 0x0000000304800000, 0x0000000304800000|100%| O| |TAMS 0x0000000304800000, 0x0000000304800000| Untracked +| 32|0x0000000304800000, 0x0000000304a00000, 0x0000000304a00000|100%| O| |TAMS 0x0000000304a00000, 0x0000000304a00000| Untracked +| 33|0x0000000304a00000, 0x0000000304c00000, 0x0000000304c00000|100%| O| |TAMS 0x0000000304c00000, 0x0000000304c00000| Untracked +| 34|0x0000000304c00000, 0x0000000304e00000, 0x0000000304e00000|100%| O| |TAMS 0x0000000304e00000, 0x0000000304e00000| Untracked +| 35|0x0000000304e00000, 0x0000000305000000, 0x0000000305000000|100%| O| |TAMS 0x0000000305000000, 0x0000000305000000| Untracked +| 36|0x0000000305000000, 0x0000000305200000, 0x0000000305200000|100%| O| |TAMS 0x0000000305200000, 0x0000000305200000| Untracked +| 37|0x0000000305200000, 0x0000000305400000, 0x0000000305400000|100%| O| |TAMS 0x0000000305400000, 0x0000000305400000| Untracked +| 38|0x0000000305400000, 0x0000000305600000, 0x0000000305600000|100%| O| |TAMS 0x0000000305600000, 0x0000000305600000| Untracked +| 39|0x0000000305600000, 0x0000000305800000, 0x0000000305800000|100%| O| |TAMS 0x0000000305800000, 0x0000000305800000| Untracked +| 40|0x0000000305800000, 0x0000000305a00000, 0x0000000305a00000|100%| O| |TAMS 0x0000000305a00000, 0x0000000305a00000| Untracked +| 41|0x0000000305a00000, 0x0000000305c00000, 0x0000000305c00000|100%| O| |TAMS 0x0000000305c00000, 0x0000000305c00000| Untracked +| 42|0x0000000305c00000, 0x0000000305d80a00, 0x0000000305e00000| 75%| O| |TAMS 0x0000000305d80a00, 0x0000000305d80a00| Untracked +| 43|0x0000000305e00000, 0x0000000306000000, 0x0000000306000000|100%|HS| |TAMS 0x0000000305e00000, 0x0000000306000000| Complete +| 44|0x0000000306000000, 0x0000000306200000, 0x0000000306200000|100%|HS| |TAMS 0x0000000306000000, 0x0000000306200000| Complete +| 45|0x0000000306200000, 0x0000000306400000, 0x0000000306400000|100%|HS| |TAMS 0x0000000306200000, 0x0000000306400000| Complete +| 46|0x0000000306400000, 0x0000000306600000, 0x0000000306600000|100%|HS| |TAMS 0x0000000306400000, 0x0000000306600000| Complete +| 47|0x0000000306600000, 0x0000000306800000, 0x0000000306800000|100%|HS| |TAMS 0x0000000306600000, 0x0000000306800000| Complete +| 48|0x0000000306800000, 0x0000000306a00000, 0x0000000306a00000|100%|HS| |TAMS 0x0000000306800000, 0x0000000306a00000| Complete +| 49|0x0000000306a00000, 0x0000000306c00000, 0x0000000306c00000|100%|HS| |TAMS 0x0000000306a00000, 0x0000000306c00000| Complete +| 50|0x0000000306c00000, 0x0000000306e00000, 0x0000000306e00000|100%|HS| |TAMS 0x0000000306c00000, 0x0000000306e00000| Complete +| 51|0x0000000306e00000, 0x0000000307000000, 0x0000000307000000|100%|HS| |TAMS 0x0000000306e00000, 0x0000000307000000| Complete +| 52|0x0000000307000000, 0x0000000307200000, 0x0000000307200000|100%|HS| |TAMS 0x0000000307000000, 0x0000000307200000| Complete +| 53|0x0000000307200000, 0x0000000307400000, 0x0000000307400000|100%| O| |TAMS 0x0000000307200000, 0x0000000307400000| Untracked +| 54|0x0000000307400000, 0x0000000307600000, 0x0000000307600000|100%| O| |TAMS 0x0000000307400000, 0x0000000307600000| Untracked +| 55|0x0000000307600000, 0x0000000307800000, 0x0000000307800000|100%| O| |TAMS 0x0000000307600000, 0x0000000307800000| Untracked +| 56|0x0000000307800000, 0x0000000307a00000, 0x0000000307a00000|100%| O| |TAMS 0x0000000307800000, 0x0000000307a00000| Untracked +| 57|0x0000000307a00000, 0x0000000307c00000, 0x0000000307c00000|100%| O| |TAMS 0x0000000307a00000, 0x0000000307c00000| Untracked +| 58|0x0000000307c00000, 0x0000000307e00000, 0x0000000307e00000|100%| O| |TAMS 0x0000000307c00000, 0x0000000307e00000| Untracked +| 59|0x0000000307e00000, 0x0000000308000000, 0x0000000308000000|100%| O| |TAMS 0x0000000307e00000, 0x0000000308000000| Untracked +| 60|0x0000000308000000, 0x00000003081e9c00, 0x0000000308200000| 95%| O| |TAMS 0x0000000308000000, 0x00000003081e9c00| Untracked +| 61|0x0000000308200000, 0x0000000308200000, 0x0000000308400000| 0%| F| |TAMS 0x0000000308200000, 0x0000000308200000| Untracked +| 62|0x0000000308400000, 0x0000000308400000, 0x0000000308600000| 0%| F| |TAMS 0x0000000308400000, 0x0000000308400000| Untracked +| 63|0x0000000308600000, 0x0000000308800000, 0x0000000308800000|100%| O| |TAMS 0x0000000308600000, 0x0000000308800000| Untracked +| 64|0x0000000308800000, 0x0000000308a00000, 0x0000000308a00000|100%| O| |TAMS 0x0000000308800000, 0x0000000308a00000| Untracked +| 65|0x0000000308a00000, 0x0000000308c00000, 0x0000000308c00000|100%| O| |TAMS 0x0000000308a00000, 0x0000000308c00000| Untracked +| 66|0x0000000308c00000, 0x0000000308e00000, 0x0000000308e00000|100%|HS| |TAMS 0x0000000308c00000, 0x0000000308c00000| Complete +| 67|0x0000000308e00000, 0x0000000309000000, 0x0000000309000000|100%|HC| |TAMS 0x0000000308e00000, 0x0000000308e00000| Complete +| 68|0x0000000309000000, 0x0000000309200000, 0x0000000309200000|100%|HC| |TAMS 0x0000000309000000, 0x0000000309000000| Complete +| 69|0x0000000309200000, 0x0000000309200000, 0x0000000309400000| 0%| F| |TAMS 0x0000000309200000, 0x0000000309200000| Untracked +| 70|0x0000000309400000, 0x0000000309400000, 0x0000000309600000| 0%| F| |TAMS 0x0000000309400000, 0x0000000309400000| Untracked +| 71|0x0000000309600000, 0x0000000309800000, 0x0000000309800000|100%| O| |TAMS 0x0000000309600000, 0x0000000309800000| Untracked +| 72|0x0000000309800000, 0x0000000309a00000, 0x0000000309a00000|100%| O| |TAMS 0x0000000309800000, 0x0000000309a00000| Untracked +| 73|0x0000000309a00000, 0x0000000309c00000, 0x0000000309c00000|100%| O| |TAMS 0x0000000309a00000, 0x0000000309c00000| Untracked +| 74|0x0000000309c00000, 0x0000000309e00000, 0x0000000309e00000|100%| O| |TAMS 0x0000000309c00000, 0x0000000309e00000| Untracked +| 75|0x0000000309e00000, 0x000000030a000000, 0x000000030a000000|100%| O| |TAMS 0x0000000309e00000, 0x000000030a000000| Untracked +| 76|0x000000030a000000, 0x000000030a200000, 0x000000030a200000|100%| O| |TAMS 0x000000030a000000, 0x000000030a200000| Untracked +| 77|0x000000030a200000, 0x000000030a400000, 0x000000030a400000|100%| O| |TAMS 0x000000030a200000, 0x000000030a400000| Untracked +| 78|0x000000030a400000, 0x000000030a600000, 0x000000030a600000|100%| O| |TAMS 0x000000030a400000, 0x000000030a600000| Untracked +| 79|0x000000030a600000, 0x000000030a600000, 0x000000030a800000| 0%| F| |TAMS 0x000000030a600000, 0x000000030a600000| Untracked +| 80|0x000000030a800000, 0x000000030a800000, 0x000000030aa00000| 0%| F| |TAMS 0x000000030a800000, 0x000000030a800000| Untracked +| 81|0x000000030aa00000, 0x000000030aa00000, 0x000000030ac00000| 0%| F| |TAMS 0x000000030aa00000, 0x000000030aa00000| Untracked +| 82|0x000000030ac00000, 0x000000030ac00000, 0x000000030ae00000| 0%| F| |TAMS 0x000000030ac00000, 0x000000030ac00000| Untracked +| 83|0x000000030ae00000, 0x000000030ae00000, 0x000000030b000000| 0%| F| |TAMS 0x000000030ae00000, 0x000000030ae00000| Untracked +| 84|0x000000030b000000, 0x000000030b000000, 0x000000030b200000| 0%| F| |TAMS 0x000000030b000000, 0x000000030b000000| Untracked +| 85|0x000000030b200000, 0x000000030b200000, 0x000000030b400000| 0%| F| |TAMS 0x000000030b200000, 0x000000030b200000| Untracked +| 86|0x000000030b400000, 0x000000030b400000, 0x000000030b600000| 0%| F| |TAMS 0x000000030b400000, 0x000000030b400000| Untracked +| 87|0x000000030b600000, 0x000000030b600000, 0x000000030b800000| 0%| F| |TAMS 0x000000030b600000, 0x000000030b600000| Untracked +| 88|0x000000030b800000, 0x000000030b800000, 0x000000030ba00000| 0%| F| |TAMS 0x000000030b800000, 0x000000030b800000| Untracked +| 89|0x000000030ba00000, 0x000000030ba00000, 0x000000030bc00000| 0%| F| |TAMS 0x000000030ba00000, 0x000000030ba00000| Untracked +| 90|0x000000030bc00000, 0x000000030bc00000, 0x000000030be00000| 0%| F| |TAMS 0x000000030bc00000, 0x000000030bc00000| Untracked +| 91|0x000000030be00000, 0x000000030be00000, 0x000000030c000000| 0%| F| |TAMS 0x000000030be00000, 0x000000030be00000| Untracked +| 92|0x000000030c000000, 0x000000030c000000, 0x000000030c200000| 0%| F| |TAMS 0x000000030c000000, 0x000000030c000000| Untracked +| 93|0x000000030c200000, 0x000000030c200000, 0x000000030c400000| 0%| F| |TAMS 0x000000030c200000, 0x000000030c200000| Untracked +| 94|0x000000030c400000, 0x000000030c400000, 0x000000030c600000| 0%| F| |TAMS 0x000000030c400000, 0x000000030c400000| Untracked +| 95|0x000000030c600000, 0x000000030c600000, 0x000000030c800000| 0%| F| |TAMS 0x000000030c600000, 0x000000030c600000| Untracked +| 96|0x000000030c800000, 0x000000030c800000, 0x000000030ca00000| 0%| F| |TAMS 0x000000030c800000, 0x000000030c800000| Untracked +| 97|0x000000030ca00000, 0x000000030ca00000, 0x000000030cc00000| 0%| F| |TAMS 0x000000030ca00000, 0x000000030ca00000| Untracked +| 98|0x000000030cc00000, 0x000000030cc00000, 0x000000030ce00000| 0%| F| |TAMS 0x000000030cc00000, 0x000000030cc00000| Untracked +| 99|0x000000030ce00000, 0x000000030ce00000, 0x000000030d000000| 0%| F| |TAMS 0x000000030ce00000, 0x000000030ce00000| Untracked +| 100|0x000000030d000000, 0x000000030d000000, 0x000000030d200000| 0%| F| |TAMS 0x000000030d000000, 0x000000030d000000| Untracked +| 101|0x000000030d200000, 0x000000030d200000, 0x000000030d400000| 0%| F| |TAMS 0x000000030d200000, 0x000000030d200000| Untracked +| 102|0x000000030d400000, 0x000000030d400000, 0x000000030d600000| 0%| F| |TAMS 0x000000030d400000, 0x000000030d400000| Untracked +| 103|0x000000030d600000, 0x000000030d600000, 0x000000030d800000| 0%| F| |TAMS 0x000000030d600000, 0x000000030d600000| Untracked +| 104|0x000000030d800000, 0x000000030d800000, 0x000000030da00000| 0%| F| |TAMS 0x000000030d800000, 0x000000030d800000| Untracked +| 105|0x000000030da00000, 0x000000030da00000, 0x000000030dc00000| 0%| F| |TAMS 0x000000030da00000, 0x000000030da00000| Untracked +| 106|0x000000030dc00000, 0x000000030dc00000, 0x000000030de00000| 0%| F| |TAMS 0x000000030dc00000, 0x000000030dc00000| Untracked +| 107|0x000000030de00000, 0x000000030de00000, 0x000000030e000000| 0%| F| |TAMS 0x000000030de00000, 0x000000030de00000| Untracked +| 108|0x000000030e000000, 0x000000030e000000, 0x000000030e200000| 0%| F| |TAMS 0x000000030e000000, 0x000000030e000000| Untracked +| 109|0x000000030e200000, 0x000000030e200000, 0x000000030e400000| 0%| F| |TAMS 0x000000030e200000, 0x000000030e200000| Untracked +| 110|0x000000030e400000, 0x000000030e400000, 0x000000030e600000| 0%| F| |TAMS 0x000000030e400000, 0x000000030e400000| Untracked +| 111|0x000000030e600000, 0x000000030e600000, 0x000000030e800000| 0%| F| |TAMS 0x000000030e600000, 0x000000030e600000| Untracked +| 112|0x000000030e800000, 0x000000030e800000, 0x000000030ea00000| 0%| F| |TAMS 0x000000030e800000, 0x000000030e800000| Untracked +| 113|0x000000030ea00000, 0x000000030ea00000, 0x000000030ec00000| 0%| F| |TAMS 0x000000030ea00000, 0x000000030ea00000| Untracked +| 114|0x000000030ec00000, 0x000000030ec00000, 0x000000030ee00000| 0%| F| |TAMS 0x000000030ec00000, 0x000000030ec00000| Untracked +| 115|0x000000030ee00000, 0x000000030ee00000, 0x000000030f000000| 0%| F| |TAMS 0x000000030ee00000, 0x000000030ee00000| Untracked +| 116|0x000000030f000000, 0x000000030f000000, 0x000000030f200000| 0%| F| |TAMS 0x000000030f000000, 0x000000030f000000| Untracked +| 117|0x000000030f200000, 0x000000030f200000, 0x000000030f400000| 0%| F| |TAMS 0x000000030f200000, 0x000000030f200000| Untracked +| 118|0x000000030f400000, 0x000000030f400000, 0x000000030f600000| 0%| F| |TAMS 0x000000030f400000, 0x000000030f400000| Untracked +| 119|0x000000030f600000, 0x000000030f600000, 0x000000030f800000| 0%| F| |TAMS 0x000000030f600000, 0x000000030f600000| Untracked +| 120|0x000000030f800000, 0x000000030f800000, 0x000000030fa00000| 0%| F| |TAMS 0x000000030f800000, 0x000000030f800000| Untracked +| 121|0x000000030fa00000, 0x000000030fa00000, 0x000000030fc00000| 0%| F| |TAMS 0x000000030fa00000, 0x000000030fa00000| Untracked +| 122|0x000000030fc00000, 0x000000030fc00000, 0x000000030fe00000| 0%| F| |TAMS 0x000000030fc00000, 0x000000030fc00000| Untracked +| 123|0x000000030fe00000, 0x000000030fe00000, 0x0000000310000000| 0%| F| |TAMS 0x000000030fe00000, 0x000000030fe00000| Untracked +| 124|0x0000000310000000, 0x0000000310000000, 0x0000000310200000| 0%| F| |TAMS 0x0000000310000000, 0x0000000310000000| Untracked +| 125|0x0000000310200000, 0x0000000310200000, 0x0000000310400000| 0%| F| |TAMS 0x0000000310200000, 0x0000000310200000| Untracked +| 126|0x0000000310400000, 0x0000000310400000, 0x0000000310600000| 0%| F| |TAMS 0x0000000310400000, 0x0000000310400000| Untracked +| 127|0x0000000310600000, 0x0000000310600000, 0x0000000310800000| 0%| F| |TAMS 0x0000000310600000, 0x0000000310600000| Untracked +| 128|0x0000000310800000, 0x0000000310800000, 0x0000000310a00000| 0%| F| |TAMS 0x0000000310800000, 0x0000000310800000| Untracked +| 129|0x0000000310a00000, 0x0000000310a00000, 0x0000000310c00000| 0%| F| |TAMS 0x0000000310a00000, 0x0000000310a00000| Untracked +| 130|0x0000000310c00000, 0x0000000310c00000, 0x0000000310e00000| 0%| F| |TAMS 0x0000000310c00000, 0x0000000310c00000| Untracked +| 131|0x0000000310e00000, 0x0000000310e00000, 0x0000000311000000| 0%| F| |TAMS 0x0000000310e00000, 0x0000000310e00000| Untracked +| 132|0x0000000311000000, 0x0000000311000000, 0x0000000311200000| 0%| F| |TAMS 0x0000000311000000, 0x0000000311000000| Untracked +| 133|0x0000000311200000, 0x0000000311200000, 0x0000000311400000| 0%| F| |TAMS 0x0000000311200000, 0x0000000311200000| Untracked +| 134|0x0000000311400000, 0x0000000311400000, 0x0000000311600000| 0%| F| |TAMS 0x0000000311400000, 0x0000000311400000| Untracked +| 135|0x0000000311600000, 0x0000000311600000, 0x0000000311800000| 0%| F| |TAMS 0x0000000311600000, 0x0000000311600000| Untracked +| 136|0x0000000311800000, 0x0000000311800000, 0x0000000311a00000| 0%| F| |TAMS 0x0000000311800000, 0x0000000311800000| Untracked +| 137|0x0000000311a00000, 0x0000000311a00000, 0x0000000311c00000| 0%| F| |TAMS 0x0000000311a00000, 0x0000000311a00000| Untracked +| 138|0x0000000311c00000, 0x0000000311c00000, 0x0000000311e00000| 0%| F| |TAMS 0x0000000311c00000, 0x0000000311c00000| Untracked +| 139|0x0000000311e00000, 0x0000000311e00000, 0x0000000312000000| 0%| F| |TAMS 0x0000000311e00000, 0x0000000311e00000| Untracked +| 140|0x0000000312000000, 0x0000000312000000, 0x0000000312200000| 0%| F| |TAMS 0x0000000312000000, 0x0000000312000000| Untracked +| 141|0x0000000312200000, 0x0000000312200000, 0x0000000312400000| 0%| F| |TAMS 0x0000000312200000, 0x0000000312200000| Untracked +| 142|0x0000000312400000, 0x0000000312400000, 0x0000000312600000| 0%| F| |TAMS 0x0000000312400000, 0x0000000312400000| Untracked +| 143|0x0000000312600000, 0x0000000312600000, 0x0000000312800000| 0%| F| |TAMS 0x0000000312600000, 0x0000000312600000| Untracked +| 144|0x0000000312800000, 0x0000000312800000, 0x0000000312a00000| 0%| F| |TAMS 0x0000000312800000, 0x0000000312800000| Untracked +| 145|0x0000000312a00000, 0x0000000312a00000, 0x0000000312c00000| 0%| F| |TAMS 0x0000000312a00000, 0x0000000312a00000| Untracked +| 146|0x0000000312c00000, 0x0000000312c00000, 0x0000000312e00000| 0%| F| |TAMS 0x0000000312c00000, 0x0000000312c00000| Untracked +| 147|0x0000000312e00000, 0x0000000312e00000, 0x0000000313000000| 0%| F| |TAMS 0x0000000312e00000, 0x0000000312e00000| Untracked +| 148|0x0000000313000000, 0x0000000313000000, 0x0000000313200000| 0%| F| |TAMS 0x0000000313000000, 0x0000000313000000| Untracked +| 149|0x0000000313200000, 0x0000000313200000, 0x0000000313400000| 0%| F| |TAMS 0x0000000313200000, 0x0000000313200000| Untracked +| 150|0x0000000313400000, 0x0000000313400000, 0x0000000313600000| 0%| F| |TAMS 0x0000000313400000, 0x0000000313400000| Untracked +| 151|0x0000000313600000, 0x0000000313600000, 0x0000000313800000| 0%| F| |TAMS 0x0000000313600000, 0x0000000313600000| Untracked +| 152|0x0000000313800000, 0x00000003139375d0, 0x0000000313a00000| 60%| S|CS|TAMS 0x0000000313800000, 0x0000000313800000| Complete +| 153|0x0000000313a00000, 0x0000000313c00000, 0x0000000313c00000|100%| S|CS|TAMS 0x0000000313a00000, 0x0000000313a00000| Complete +| 154|0x0000000313c00000, 0x0000000313e00000, 0x0000000313e00000|100%| S|CS|TAMS 0x0000000313c00000, 0x0000000313c00000| Complete +| 155|0x0000000313e00000, 0x0000000314000000, 0x0000000314000000|100%| S|CS|TAMS 0x0000000313e00000, 0x0000000313e00000| Complete +| 156|0x0000000314000000, 0x0000000314000000, 0x0000000314200000| 0%| F| |TAMS 0x0000000314000000, 0x0000000314000000| Untracked +| 157|0x0000000314200000, 0x0000000314400000, 0x0000000314400000|100%| E| |TAMS 0x0000000314200000, 0x0000000314200000| Complete +| 158|0x0000000314400000, 0x0000000314600000, 0x0000000314600000|100%| E|CS|TAMS 0x0000000314400000, 0x0000000314400000| Complete +| 159|0x0000000314600000, 0x0000000314800000, 0x0000000314800000|100%| E|CS|TAMS 0x0000000314600000, 0x0000000314600000| Complete +| 160|0x0000000314800000, 0x0000000314a00000, 0x0000000314a00000|100%| E|CS|TAMS 0x0000000314800000, 0x0000000314800000| Complete +| 161|0x0000000314a00000, 0x0000000314c00000, 0x0000000314c00000|100%| E|CS|TAMS 0x0000000314a00000, 0x0000000314a00000| Complete +| 335|0x000000032a600000, 0x000000032a800000, 0x000000032a800000|100%| E|CS|TAMS 0x000000032a600000, 0x000000032a600000| Complete +| 336|0x000000032a800000, 0x000000032aa00000, 0x000000032aa00000|100%| E|CS|TAMS 0x000000032a800000, 0x000000032a800000| Complete +| 440|0x0000000337800000, 0x0000000337a00000, 0x0000000337a00000|100%| E|CS|TAMS 0x0000000337800000, 0x0000000337800000| Complete +| 441|0x0000000337a00000, 0x0000000337c00000, 0x0000000337c00000|100%| E|CS|TAMS 0x0000000337a00000, 0x0000000337a00000| Complete +| 442|0x0000000337c00000, 0x0000000337e00000, 0x0000000337e00000|100%| E|CS|TAMS 0x0000000337c00000, 0x0000000337c00000| Complete +| 767|0x0000000360600000, 0x0000000360800000, 0x0000000360800000|100%| E|CS|TAMS 0x0000000360600000, 0x0000000360600000| Complete +|2046|0x0000000400400000, 0x0000000400574000, 0x0000000400600000| 72%|OA| |TAMS 0x0000000400574000, 0x0000000400574000| Untracked +|2047|0x0000000400600000, 0x0000000400674000, 0x0000000400800000| 22%|CA| |TAMS 0x0000000400674000, 0x0000000400674000| Untracked + +Card table byte_map: [0x000000011e410000,0x000000011ec10000] _byte_map_base: 0x000000011cc0c000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x00000001014de750, (CMBitMap*) 0x00000001014de790 + Prev Bits: [0x000000011f410000, 0x0000000123410000) + Next Bits: [0x0000000125e04000, 0x0000000129e04000) + +Polling page: 0x0000000100dc4000 + +Metaspace: + +Usage: + Non-class: 130.17 MB used. + Class: 17.79 MB used. + Both: 147.96 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 130.88 MB ( 68%) committed, 3 nodes. + Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 149.25 MB ( 12%) committed. + +Chunk freelists: + Non-Class: 12.93 MB + Class: 13.67 MB + Both: 26.61 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 164.38 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 14. +num_arena_births: 2116. +num_arena_deaths: 0. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2385. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 14. +num_chunks_taken_from_freelist: 7831. +num_chunk_merges: 11. +num_chunk_splits: 5823. +num_chunks_enlarged: 4407. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=8762Kb max_used=9288Kb free=110405Kb + bounds [0x0000000116fb0000, 0x00000001178d0000, 0x000000011e410000] +CodeHeap 'profiled nmethods': size=119152Kb used=19803Kb max_used=20506Kb free=99349Kb + bounds [0x000000010f410000, 0x0000000110820000, 0x000000011686c000] +CodeHeap 'non-nmethods': size=7440Kb used=3322Kb max_used=3496Kb free=4117Kb + bounds [0x000000011686c000, 0x0000000116bec000, 0x0000000116fb0000] + total_blobs=11680 nmethods=10618 adapters=973 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 21.992 Thread 0x0000000758d82400 nmethod 14784 0x00000001102ee910 code [0x00000001102eeac0, 0x00000001102eecf8] +Event: 21.992 Thread 0x0000000758d82400 14790 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) +Event: 21.992 Thread 0x0000000752b5de00 nmethod 14788 0x00000001102ee290 code [0x00000001102ee480, 0x00000001102ee738] +Event: 21.992 Thread 0x0000000752b5de00 14792 3 org.apache.logging.slf4j.Log4jLogger::info (27 bytes) +Event: 21.992 Thread 0x0000000758d82400 nmethod 14790 0x00000001102edc90 code [0x00000001102ede80, 0x00000001102ee0e8] +Event: 21.992 Thread 0x0000000758d82400 14791 3 org.apache.logging.log4j.message.SimpleMessage::formatTo (43 bytes) +Event: 21.992 Thread 0x0000000752b5de00 nmethod 14792 0x000000010fee0a10 code [0x000000010fee0c00, 0x000000010fee0eb8] +Event: 21.992 Thread 0x0000000752b5de00 14793 3 org.apache.comet.ConfigEntry::key (13 bytes) +Event: 21.992 Thread 0x0000000752b5de00 nmethod 14793 0x000000010fee0590 code [0x000000010fee0740, 0x000000010fee08c8] +Event: 21.992 Thread 0x0000000752b5de00 14794 3 java.util.GregorianCalendar::computeFields (85 bytes) +Event: 21.992 Thread 0x0000000758d82400 nmethod 14791 0x000000010fedfe90 code [0x000000010fee0080, 0x000000010fee03b8] +Event: 21.992 Thread 0x0000000752b5de00 nmethod 14794 0x000000010fedf510 code [0x000000010fedf740, 0x000000010fedfb98] +Event: 21.993 Thread 0x0000000752b5d800 nmethod 14787 0x000000010f4d1a90 code [0x000000010f4d1e80, 0x000000010f4d3108] +Event: 21.993 Thread 0x0000000758d82400 14795 3 scala.collection.convert.Wrappers$IteratorWrapper::$jacocoInit (52 bytes) +Event: 21.993 Thread 0x0000000752b5e400 nmethod 14789 0x0000000110476910 code [0x0000000110476fc0, 0x000000011047a3e8] +Event: 21.993 Thread 0x0000000758d82400 nmethod 14795 0x0000000110439f90 code [0x000000011043a200, 0x000000011043abb8] +Event: 21.993 Thread 0x0000000752b5de00 14796 3 org.apache.comet.CometExecIterator$$$Lambda$3896/0x000000f8021abc98::apply (12 bytes) +Event: 21.993 Thread 0x0000000752b5d800 14797 3 org.apache.comet.CometExecIterator$::$anonfun$serializeCometSQLConfs$1 (159 bytes) +Event: 21.993 Thread 0x0000000752b5de00 nmethod 14796 0x0000000110439a90 code [0x0000000110439c40, 0x0000000110439e88] +Event: 21.993 Thread 0x0000000758d82400 14798 3 com.google.protobuf.MapEntry::writeTo (26 bytes) + +GC Heap History (20 events): +Event: 7.549 GC heap before +{Heap before GC invocations=28 (full 0): + garbage-first heap total 348160K, used 284481K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 66 young (135168K), 2 survivors (4096K) + Metaspace used 140778K, committed 141760K, reserved 1179648K + class space used 16878K, committed 17344K, reserved 1048576K +} +Event: 7.551 GC heap after +{Heap after GC invocations=29 (full 0): + garbage-first heap total 348160K, used 152858K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 140778K, committed 141760K, reserved 1179648K + class space used 16878K, committed 17344K, reserved 1048576K +} +Event: 7.727 GC heap before +{Heap before GC invocations=29 (full 0): + garbage-first heap total 348160K, used 290074K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 68 young (139264K), 3 survivors (6144K) + Metaspace used 147454K, committed 148544K, reserved 1179648K + class space used 17571K, committed 18048K, reserved 1048576K +} +Event: 7.728 GC heap after +{Heap after GC invocations=30 (full 0): + garbage-first heap total 348160K, used 154595K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 147454K, committed 148544K, reserved 1179648K + class space used 17571K, committed 18048K, reserved 1048576K +} +Event: 8.064 GC heap before +{Heap before GC invocations=30 (full 0): + garbage-first heap total 348160K, used 289763K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 70 young (143360K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.066 GC heap after +{Heap after GC invocations=31 (full 0): + garbage-first heap total 348160K, used 154934K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.335 GC heap before +{Heap before GC invocations=31 (full 0): + garbage-first heap total 348160K, used 292150K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 71 young (145408K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.337 GC heap after +{Heap after GC invocations=32 (full 0): + garbage-first heap total 348160K, used 155446K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.625 GC heap before +{Heap before GC invocations=32 (full 0): + garbage-first heap total 348160K, used 296758K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 73 young (149504K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.626 GC heap after +{Heap after GC invocations=33 (full 0): + garbage-first heap total 348160K, used 155007K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.917 GC heap before +{Heap before GC invocations=33 (full 0): + garbage-first heap total 348160K, used 298367K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 74 young (151552K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 8.918 GC heap after +{Heap after GC invocations=34 (full 0): + garbage-first heap total 348160K, used 155261K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 9.204 GC heap before +{Heap before GC invocations=34 (full 0): + garbage-first heap total 348160K, used 300669K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 75 young (153600K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 9.206 GC heap after +{Heap after GC invocations=35 (full 0): + garbage-first heap total 348160K, used 155102K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148866K, committed 150016K, reserved 1245184K + class space used 17780K, committed 18304K, reserved 1048576K +} +Event: 21.867 GC heap before +{Heap before GC invocations=35 (full 0): + garbage-first heap total 348160K, used 298462K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 73 young (149504K), 4 survivors (8192K) + Metaspace used 149284K, committed 150400K, reserved 1245184K + class space used 17854K, committed 18368K, reserved 1048576K +} +Event: 21.869 GC heap after +{Heap after GC invocations=36 (full 0): + garbage-first heap total 348160K, used 155067K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 149284K, committed 150400K, reserved 1245184K + class space used 17854K, committed 18368K, reserved 1048576K +} +Event: 21.905 GC heap before +{Heap before GC invocations=36 (full 0): + garbage-first heap total 348160K, used 161211K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 6 young (12288K), 4 survivors (8192K) + Metaspace used 149340K, committed 150464K, reserved 1245184K + class space used 17854K, committed 18368K, reserved 1048576K +} +Event: 21.907 GC heap after +{Heap after GC invocations=37 (full 0): + garbage-first heap total 348160K, used 155089K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 149340K, committed 150464K, reserved 1245184K + class space used 17854K, committed 18368K, reserved 1048576K +} +Event: 21.957 GC heap before +{Heap before GC invocations=37 (full 0): + garbage-first heap total 348160K, used 171473K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 12 young (24576K), 4 survivors (8192K) + Metaspace used 150237K, committed 151424K, reserved 1245184K + class space used 17984K, committed 18496K, reserved 1048576K +} +Event: 21.959 GC heap after +{Heap after GC invocations=38 (full 0): + garbage-first heap total 348160K, used 155166K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 150237K, committed 151424K, reserved 1245184K + class space used 17984K, committed 18496K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.514 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li +Event: 3.514 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.514 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.575 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14714718128898053499.dylib +Event: 6.098 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 6.098 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 6.098 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 6.098 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 6.098 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 6.098 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 6.098 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 6.098 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 6.098 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 6.098 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 6.098 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 6.098 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 6.098 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 6.098 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 6.219 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-12903721131319273091.dylib +Event: 6.983 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-6a22215f-8250-484c-aa43-855ee20024fe-libsnappyjava.dylib + +Deoptimization events (20 events): +Event: 21.961 Thread 0x000000075839de00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x0000000117807de4 relative=0x00000000000006a4 +Event: 21.961 Thread 0x000000075839de00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x0000000117807de4 method=scala.runtime.BoxesRunTime.equalsNumNum(Ljava/lang/Number;Ljava/lang/Number;)Z @ 84 c2 +Event: 21.961 Thread 0x000000075839de00 DEOPT PACKING pc=0x0000000117807de4 sp=0x000000017cb756e0 +Event: 21.961 Thread 0x000000075839de00 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000017cb754f0 mode 2 +Event: 21.961 Thread 0x000000075839de00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x0000000117809410 relative=0x0000000000000650 +Event: 21.961 Thread 0x000000075839de00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x0000000117809410 method=scala.runtime.BoxesRunTime.equalsNumNum(Ljava/lang/Number;Ljava/lang/Number;)Z @ 84 c2 +Event: 21.961 Thread 0x000000075839de00 DEOPT PACKING pc=0x0000000117809410 sp=0x000000017cb755c0 +Event: 21.961 Thread 0x000000075839de00 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000017cb75440 mode 2 +Event: 21.969 Thread 0x0000000755abd800 Uncommon trap: trap_request=0xfffffff4 fr.pc=0x00000001175d999c relative=0x000000000000045c +Event: 21.969 Thread 0x0000000755abd800 Uncommon trap: reason=null_check action=make_not_entrant pc=0x00000001175d999c method=java.io.ObjectInputStream.readArray(Z)Ljava/lang/Object; @ 392 c2 +Event: 21.969 Thread 0x0000000755abd800 DEOPT PACKING pc=0x00000001175d999c sp=0x0000000401010070 +Event: 21.969 Thread 0x0000000755abd800 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000040100ffd0 mode 2 +Event: 21.982 Thread 0x0000000755abe400 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011785ab90 relative=0x0000000000000810 +Event: 21.982 Thread 0x0000000755abe400 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011785ab90 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 +Event: 21.982 Thread 0x0000000755abe400 DEOPT PACKING pc=0x000000011785ab90 sp=0x0000000401828830 +Event: 21.982 Thread 0x0000000755abe400 DEOPT UNPACKING pc=0x00000001168b391c sp=0x00000004018286a0 mode 2 +Event: 21.987 Thread 0x0000000755abe400 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011785ab90 relative=0x0000000000000810 +Event: 21.987 Thread 0x0000000755abe400 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011785ab90 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 +Event: 21.987 Thread 0x0000000755abe400 DEOPT PACKING pc=0x000000011785ab90 sp=0x0000000401828850 +Event: 21.987 Thread 0x0000000755abe400 DEOPT UNPACKING pc=0x00000001168b391c sp=0x00000004018286a0 mode 2 + +Classes loaded (20 events): +Event: 7.699 Loading class java/lang/Class$EnclosingMethodInfo +Event: 7.699 Loading class java/lang/Class$EnclosingMethodInfo done +Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper +Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$Unmapper +Event: 16.706 Loading class jdk/internal/access/foreign/UnmapperProxy +Event: 16.706 Loading class jdk/internal/access/foreign/UnmapperProxy done +Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$Unmapper done +Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done +Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask +Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask +Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done +Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done +Event: 19.932 Loading class java/util/function/LongBinaryOperator +Event: 19.932 Loading class java/util/function/LongBinaryOperator done +Event: 21.793 Loading class sun/nio/ch/ChannelInputStream +Event: 21.794 Loading class sun/nio/ch/ChannelInputStream done +Event: 21.922 Loading class java/math/BigDecimal$StringBuilderHelper +Event: 21.922 Loading class java/math/BigDecimal$StringBuilderHelper done +Event: 21.988 Loading class java/nio/channels/Channels$ReadableByteChannelImpl +Event: 21.988 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 21.984 Thread 0x0000000755abde00 Exception (0x0000000314958d78) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.984 Thread 0x0000000755abe400 Exception (0x00000003149ca8c8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.984 Thread 0x0000000755abd800 Exception (0x0000000314be81c8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.984 Thread 0x0000000755abd200 Exception (0x0000000314b5d518) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.984 Thread 0x0000000755abea00 Exception (0x00000003148ebf68) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c18c8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b72188) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314711de8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314bfc940) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abde00 Exception (0x000000031496cf48) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c58b8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b75f60) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abde00 Exception (0x0000000314970d20) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c9198) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314715bc0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314400d70) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b797f0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abde00 Exception (0x00000003149745b0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314719450) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314404600) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] + +VM Operations (20 events): +Event: 17.246 Executing VM operation: Cleanup +Event: 17.246 Executing VM operation: Cleanup done +Event: 20.261 Executing VM operation: Cleanup +Event: 20.261 Executing VM operation: Cleanup done +Event: 21.849 Executing VM operation: HandshakeAllThreads +Event: 21.849 Executing VM operation: HandshakeAllThreads done +Event: 21.849 Executing VM operation: Cleanup +Event: 21.849 Executing VM operation: Cleanup done +Event: 21.864 Executing VM operation: HandshakeAllThreads +Event: 21.864 Executing VM operation: HandshakeAllThreads done +Event: 21.867 Executing VM operation: G1TryInitiateConcMark +Event: 21.869 Executing VM operation: G1TryInitiateConcMark done +Event: 21.895 Executing VM operation: HandshakeAllThreads +Event: 21.895 Executing VM operation: HandshakeAllThreads done +Event: 21.905 Executing VM operation: G1TryInitiateConcMark +Event: 21.907 Executing VM operation: G1TryInitiateConcMark done +Event: 21.948 Executing VM operation: ICBufferFull +Event: 21.948 Executing VM operation: ICBufferFull done +Event: 21.957 Executing VM operation: G1TryInitiateConcMark +Event: 21.959 Executing VM operation: G1TryInitiateConcMark done + +Memory protections (20 events): +Event: 7.559 Protecting memory [0x0000000403488000,0x0000000403494000] with protection modes 0 +Event: 7.559 Protecting memory [0x0000000403894000,0x00000004038a0000] with protection modes 0 +Event: 9.475 Protecting memory [0x0000000176430000,0x000000017643c000] with protection modes 3 +Event: 9.475 Protecting memory [0x0000000173734000,0x0000000173740000] with protection modes 3 +Event: 9.475 Protecting memory [0x0000000172af8000,0x0000000172b04000] with protection modes 3 +Event: 9.478 Protecting memory [0x00000001728ec000,0x00000001728f8000] with protection modes 3 +Event: 9.478 Protecting memory [0x00000001726e0000,0x00000001726ec000] with protection modes 3 +Event: 14.483 Protecting memory [0x00000001724d4000,0x00000001724e0000] with protection modes 3 +Event: 19.931 Protecting memory [0x00000001722c8000,0x00000001722d4000] with protection modes 3 +Event: 21.796 Protecting memory [0x00000001722c8000,0x00000001722d4000] with protection modes 0 +Event: 21.849 Protecting memory [0x00000001724d4000,0x00000001724e0000] with protection modes 0 +Event: 21.849 Protecting memory [0x00000001726e0000,0x00000001726ec000] with protection modes 0 +Event: 21.849 Protecting memory [0x00000001728ec000,0x00000001728f8000] with protection modes 0 +Event: 21.896 Protecting memory [0x0000000172af8000,0x0000000172b04000] with protection modes 0 +Event: 21.911 Protecting memory [0x00000004044d0000,0x00000004044dc000] with protection modes 0 +Event: 21.960 Protecting memory [0x00000004048dc000,0x00000004048e8000] with protection modes 0 +Event: 21.961 Protecting memory [0x0000000404d00000,0x0000000404d0c000] with protection modes 0 +Event: 21.961 Protecting memory [0x000000040510c000,0x0000000405118000] with protection modes 0 +Event: 21.961 Protecting memory [0x0000000405518000,0x0000000405524000] with protection modes 0 +Event: 21.961 Protecting memory [0x0000000405924000,0x0000000405930000] with protection modes 0 + +Nmethod flushes (20 events): +Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x000000011079af90 +Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x000000011079f690 +Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a5990 +Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a5e10 +Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a6490 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a9e90 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107aa490 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107aad90 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ac210 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ae910 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107afd10 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b1590 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b2c10 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b3510 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b3910 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b5e90 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ba310 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d7690 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d8510 +Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d8990 + +Events (20 events): +Event: 7.559 Thread 0x0000000752907600 Thread added: 0x0000000752974000 +Event: 7.559 Thread 0x0000000752974000 Thread added: 0x0000000752974600 +Event: 9.475 Thread 0x0000000756ff3000 Thread exited: 0x0000000756ff3000 +Event: 9.475 Thread 0x0000000755abf000 Thread exited: 0x0000000755abf000 +Event: 9.475 Thread 0x0000000755acb000 Thread exited: 0x0000000755acb000 +Event: 9.478 Thread 0x0000000754d86400 Thread exited: 0x0000000754d86400 +Event: 9.478 Thread 0x0000000754d4de00 Thread exited: 0x0000000754d4de00 +Event: 14.483 Thread 0x000000075598ea00 Thread exited: 0x000000075598ea00 +Event: 19.931 Thread 0x0000000758d83600 Thread exited: 0x0000000758d83600 +Event: 21.796 Thread 0x0000000758d82400 Thread added: 0x0000000752b5d200 +Event: 21.849 Thread 0x0000000752b5d200 Thread added: 0x0000000752b5d800 +Event: 21.849 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5de00 +Event: 21.849 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5e400 +Event: 21.896 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5ea00 +Event: 21.911 Thread 0x000000075839de00 Thread added: 0x0000000758e9f600 +Event: 21.960 Thread 0x0000000758375800 Thread added: 0x0000000755abf000 +Event: 21.961 Thread 0x0000000755abf000 Thread added: 0x0000000755abcc00 +Event: 21.961 Thread 0x0000000755abf000 Thread added: 0x0000000755acb000 +Event: 21.961 Thread 0x0000000755acb000 Thread added: 0x0000000756ff3000 +Event: 21.961 Thread 0x0000000756ff3000 Thread added: 0x00000007535b4000 + + +Dynamic libraries: +0x0000000100ddc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x00000001068cc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x0000000100e04000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x0000000100e34000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x0000000100e84000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x0000000100f20000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x0000000100f48000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x0000000100f68000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x0000000100f8c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x0000000100f0c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x0000000100fa8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x0000000100fbc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x0000000106444000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000145dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14714718128898053499.dylib +0x0000000106514000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-12903721131319273091.dylib +0x0000000106540000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-6a22215f-8250-484c-aa43-855ee20024fe-libsnappyjava.dylib + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:38 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 10.87 7.26 5.05 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(1982112k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/hs_err_pid58919.log b/spark/hs_err_pid58919.log new file mode 100644 index 0000000000..b6306fa045 --- /dev/null +++ b/spark/hs_err_pid58919.log @@ -0,0 +1,1602 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x000000010af31f64, pid=58919, tid=86283 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 08:57:06 2026 MDT elapsed time: 20.876276 seconds (0d 0h 0m 20s) + +--------------- T H R E A D --------------- + +Current thread (0x00000008d513e400): JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=86283, stack(0x000000017c130000,0x000000017c533000)] + +Stack: [0x000000017c130000,0x000000017c533000], sp=0x000000017c51a120, free space=4008k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +C [libcomet-16636361560025204640.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 +C [libcomet-16636361560025204640.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 +C [libcomet-16636361560025204640.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c +C [libcomet-16636361560025204640.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 +C [libcomet-16636361560025204640.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 +C [libcomet-16636361560025204640.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 +C [libcomet-16636361560025204640.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 +C [libcomet-16636361560025204640.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-16636361560025204640.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-16636361560025204640.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-16636361560025204640.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-16636361560025204640.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-16636361560025204640.dylib+0x106628] __rust_try+0x20 +C [libcomet-16636361560025204640.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-16636361560025204640.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-16636361560025204640.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-16636361560025204640.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3952+0x00000005021bf940.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3946+0x00000005021beb30.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2517+0x0000000501df17d0.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3952+0x00000005021bf940.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3946+0x00000005021beb30.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2517+0x0000000501df17d0.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000234b24 + +Registers: + x0=0x00000005021ae260 x1=0x0000000000000010 x2=0x00000008c6440fe0 x3=0x000000017c51f298 + x4=0x0000000000000238 x5=0x0000000000000018 x6=0x00000008c4852220 x7=0xfffff0003ffff800 + x8=0x00000000021ae260 x9=0x0000000000000000 x10=0x0000000500000000 x11=0x00000008c90485c0 +x12=0x00000008cc1300a0 x13=0x0000000000000001 x14=0x000000010539cb98 x15=0x00000008c9048000 +x16=0x000000018c8009d0 x17=0x00000008c8fcc500 x18=0x0000000000000000 x19=0x00000008d513e400 +x20=0x0000000000000001 x21=0x00000008d2a00c80 x22=0x0000000000234b10 x23=0x000000010b691088 +x24=0x0000000000000000 x25=0x000000010b695f38 x26=0x0000000138fc7948 x27=0x00000003034a7f58 +x28=0x00000008d513e400 fp=0x000000017c51a160 lr=0x000000010af31f38 sp=0x000000017c51a120 +pc=0x000000010af31f64 cpsr=0x0000000040001000 + +Register to memory mapping: + + x0=0x00000005021ae260 is a pointer to class: +org.apache.comet.CometHandleBatchIterator {0x00000005021ae260} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x0000000139041ae0) + - method ordering: Array(0x0000000500913f08) + - local interfaces: Array(0x00000005005d4fb8) + - trans. interfaces: Array(0x00000005005d4fb8) + - constants: constant pool [62]/operands[4] {0x0000000139041850} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000139041d30 + - class loader data: loader data: 0x00000008c909d0e0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x0000000303323e00} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x00000005005d4f48) + - nest members: Array(0x00000005005d4f48) + - permitted subclasses: Array(0x00000005005d4f48) + - java mirror: a 'java/lang/Class'{0x000000030b4c7778} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x00000005021ae430) + - itable length 2 (start addr: 0x00000005021ae460) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x1=0x0000000000000010 is an unknown value + x2=0x00000008c6440fe0 points into unknown readable memory: 0x000000030c2a0ba0 | a0 0b 2a 0c 03 00 00 00 + x3=0x000000017c51f298 is pointing into the stack for thread: 0x00000008d513e400 + x4=0x0000000000000238 is an unknown value + x5=0x0000000000000018 is an unknown value + x6=0x00000008c4852220 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=35316320 is a compressed pointer to class: 0x00000005021ae260 +org.apache.comet.CometHandleBatchIterator {0x00000005021ae260} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x0000000139041ae0) + - method ordering: Array(0x0000000500913f08) + - local interfaces: Array(0x00000005005d4fb8) + - trans. interfaces: Array(0x00000005005d4fb8) + - constants: constant pool [62]/operands[4] {0x0000000139041850} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000139041d30 + - class loader data: loader data: 0x00000008c909d0e0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x0000000303323e00} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x00000005005d4f48) + - nest members: Array(0x00000005005d4f48) + - permitted subclasses: Array(0x00000005005d4f48) + - java mirror: a 'java/lang/Class'{0x000000030b4c7778} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x00000005021ae430) + - itable length 2 (start addr: 0x00000005021ae460) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x9=0x0 is NULL +x10=0x0000000500000000 is pointing into metadata +x11=0x00000008c90485c0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +x12=0x00000008cc1300a0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x13=0x0000000000000001 is an unknown value +x14=0x000000010539cb98 points into unknown readable memory: 0x30eb1755804499e0 | e0 99 44 80 55 17 eb 30 +x15=0x00000008c9048000 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 +x17=0x00000008c8fcc500 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +x18=0x0 is NULL +x19=0x00000008d513e400 is a thread +x20=0x0000000000000001 is an unknown value +x21=0x00000008d2a00c80 is a global jni handle +x22=0x0000000000234b10 is an unknown value +x23=0x000000010b691088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 +x24=0x0 is NULL +x25=0x000000010b695f38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 +x26=0x0000000138fc7948 is pointing into metadata +x27=0x00000003034a7f58 is an oop: [Z +{0x00000003034a7f58} - klass: {type array bool} + - length: 9 +x28=0x00000008d513e400 is a thread + + +Top of Stack: (sp=0x000000017c51a120) +0x000000017c51a120: 0000000138fc7948 000000030334d9d0 +0x000000017c51a130: 000000017c531818 000000030334d3b0 +0x000000017c51a140: 0000000000000000 000000010b6dff90 +0x000000017c51a150: 000000017c531738 000000011aa1b72c +0x000000017c51a160: 000000017c51a5f0 000000040717ce4c +0x000000017c51a170: 0000000000000000 000000040a09e880 +0x000000017c51a180: 000000017c51a190 00000004071b6d54 +0x000000017c51a190: 00000008c6440fe0 00000008d2a00c80 +0x000000017c51a1a0: 000000010af31e08 00000008d719ccd0 +0x000000017c51a1b0: 000000010b69aad0 000000010b69aad0 +0x000000017c51a1c0: 00000008d513e6b0 00000008d513e6b0 +0x000000017c51a1d0: 000000040a0aae90 000000000717ee14 +0x000000017c51a1e0: 0000000000000000 000000017c51a270 +0x000000017c51a1f0: 00000008d513e6b0 000000040a0aae90 +0x000000017c51a200: 00000000d513e400 000000017c51a860 +0x000000017c51a210: 000000017c51aa00 8000000000000020 +0x000000017c51a220: 000000040a0aae90 00000008d719ccd0 +0x000000017c51a230: 000000040a0aae90 8000000000000020 +0x000000017c51a240: 000000040a0aae90 00000008d513e6b0 +0x000000017c51a250: 00000008d513e6b0 0000000000000002 +0x000000017c51a260: 00000008c8fcc500 0000000000000000 +0x000000017c51a270: 00000008d513e6b0 0000000000000002 +0x000000017c51a280: 00000008c8fcc500 8000000000000020 +0x000000017c51a290: 000000040a0aae90 00000008c8fcc5d0 +0x000000017c51a2a0: 000000017c51a2e0 00000004071dfc28 +0x000000017c51a2b0: 0000000000000000 0000000000000000 +0x000000017c51a2c0: 00000008d513e6b0 000000040a09e888 +0x000000017c51a2d0: 0000000000000000 00000008d513e6b0 +0x000000017c51a2e0: 00000008d513e6b0 0000000000000002 +0x000000017c51a2f0: 0000000000000008 000000040a09e888 +0x000000017c51a300: 000000017c51a340 00000004071b7d8c +0x000000017c51a310: 000000017c51a3f8 00000008d513e6b0 + +Instructions: (pc=0x000000010af31f64) +0x000000010af31e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 +0x000000010af31e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 +0x000000010af31e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 +0x000000010af31e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 +0x000000010af31ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 +0x000000010af31eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 +0x000000010af31ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 +0x000000010af31ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 +0x000000010af31ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 +0x000000010af31ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a +0x000000010af31f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 +0x000000010af31f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa +0x000000010af31f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 +0x000000010af31f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 +0x000000010af31f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 +0x000000010af31f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b +0x000000010af31f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 +0x000000010af31f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 +0x000000010af31f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 +0x000000010af31f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb +0x000000010af31fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 +0x000000010af31fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 +0x000000010af31fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 +0x000000010af31fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 +0x000000010af31fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 +0x000000010af31ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 +0x000000010af32004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b +0x000000010af32014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 +0x000000010af32024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 +0x000000010af32034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 +0x000000010af32044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 +0x000000010af32054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x0000000138fc7948 is pointing into metadata +stack at sp + 1 slots: 0x000000030334d9d0 is an oop: [Z +{0x000000030334d9d0} - klass: {type array bool} + - length: 264 +stack at sp + 2 slots: 0x000000017c531818 is pointing into the stack for thread: 0x00000008d513e400 +stack at sp + 3 slots: 0x000000030334d3b0 is an oop: [Z +{0x000000030334d3b0} - klass: {type array bool} + - length: 187 +stack at sp + 4 slots: 0x0 is NULL +stack at sp + 5 slots: 0x000000010b6dff90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 +stack at sp + 6 slots: 0x000000017c531738 is pointing into the stack for thread: 0x00000008d513e400 +stack at sp + 7 slots: 0x000000011aa1b72c is at code_begin+44 in an Interpreter codelet +result handlers for native calls [0x000000011aa1b700, 0x000000011aa1b780] 128 bytes + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x00000008c60b5f20, length=72, elements={ +0x000000010567a000, 0x000000010568e6d0, 0x00000008cc040000, 0x00000008cc040c00, +0x00000008cc041200, 0x00000008cc041800, 0x00000008cc041e00, 0x00000008cc042400, +0x00000008cc042a00, 0x00000008cc043000, 0x00000008cc17aa00, 0x00000008d27f0600, +0x00000008d27f2a00, 0x00000008d4c41800, 0x00000008d4c41200, 0x00000008d4c40c00, +0x00000008d4c42400, 0x00000008d47daa00, 0x00000008d4cc4c00, 0x00000008d4cc5200, +0x00000008d4cc5800, 0x00000008d4cc5e00, 0x00000008d4cc6400, 0x00000008d4cc6a00, +0x00000008d4cc7000, 0x00000008d4cc7600, 0x00000008d4ccde00, 0x00000008d4d00000, +0x00000008d4d00600, 0x00000008d4d01800, 0x00000008d4d01e00, 0x00000008d4d02a00, +0x00000008d4d03000, 0x00000008d4d03600, 0x00000008d4d14600, 0x00000008d4d14c00, +0x00000008d4d15e00, 0x00000008d4d16400, 0x00000008d4d17000, 0x00000008d4d28000, +0x00000008d4d28600, 0x00000008d4d28c00, 0x00000008d4d29200, 0x00000008d4d69800, +0x00000008d4d6b000, 0x00000008d4d94000, 0x00000008d4d96a00, 0x00000008d4b7c000, +0x00000008d513d800, 0x00000008d513de00, 0x00000008d513e400, 0x00000008d513ea00, +0x00000008d513f000, 0x00000008d5147600, 0x00000008ce8e2a00, 0x00000008d68d5800, +0x00000008cfaf4c00, 0x00000008d4bfea00, 0x00000008d4bfde00, 0x00000008cc179200, +0x00000008d5144000, 0x00000008d708ea00, 0x00000008d708f000, 0x00000008d708f600, +0x00000008d70e4000, 0x00000008d70e4600, 0x00000008d4bfc600, 0x00000008cc043600, +0x00000008cc179800, 0x00000008cc178000, 0x00000008d11a4c00, 0x00000008d70e7000 +} + +Java Threads: ( => current thread ) + 0x000000010567a000 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4611, stack(0x000000016afb8000,0x000000016b3bb000)] + 0x000000010568e6d0 JavaThread "Reference Handler" daemon [_thread_blocked, id=31747, stack(0x000000016c00c000,0x000000016c40f000)] + 0x00000008cc040000 JavaThread "Finalizer" daemon [_thread_blocked, id=23555, stack(0x000000016c418000,0x000000016c81b000)] + 0x00000008cc040c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30467, stack(0x000000016c93c000,0x000000016cd3f000)] + 0x00000008cc041200 JavaThread "Service Thread" daemon [_thread_blocked, id=26115, stack(0x000000016cd48000,0x000000016d14b000)] + 0x00000008cc041800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=29955, stack(0x000000016d154000,0x000000016d557000)] + 0x00000008cc041e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26627, stack(0x000000016d560000,0x000000016d763000)] + 0x00000008cc042400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27139, stack(0x000000016d76c000,0x000000016d96f000)] + 0x00000008cc042a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27395, stack(0x000000016d978000,0x000000016dd7b000)] + 0x00000008cc043000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27907, stack(0x000000016dd84000,0x000000016e187000)] + 0x00000008cc17aa00 JavaThread "Notification Thread" daemon [_thread_blocked, id=33539, stack(0x000000016edd8000,0x000000016f1db000)] + 0x00000008d27f0600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45059, stack(0x0000000171ce0000,0x00000001720e3000)] + 0x00000008d27f2a00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=63235, stack(0x0000000172b28000,0x0000000172f2b000)] + 0x00000008d4c41800 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=62979, stack(0x0000000172f34000,0x0000000173337000)] + 0x00000008d4c41200 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=62467, stack(0x0000000173340000,0x0000000173743000)] + 0x00000008d4c40c00 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=62211, stack(0x000000017374c000,0x0000000173b4f000)] + 0x00000008d4c42400 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=61699, stack(0x0000000173b58000,0x0000000173f5b000)] + 0x00000008d47daa00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=47619, stack(0x0000000173f64000,0x0000000174367000)] + 0x00000008d4cc4c00 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=48131, stack(0x0000000174370000,0x0000000174773000)] + 0x00000008d4cc5200 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=48387, stack(0x000000017477c000,0x0000000174b7f000)] + 0x00000008d4cc5800 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=48643, stack(0x0000000174b88000,0x0000000174f8b000)] + 0x00000008d4cc5e00 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=60675, stack(0x0000000174f94000,0x0000000175397000)] + 0x00000008d4cc6400 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=49155, stack(0x00000001753a0000,0x00000001757a3000)] + 0x00000008d4cc6a00 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=59907, stack(0x00000001757ac000,0x0000000175baf000)] + 0x00000008d4cc7000 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49667, stack(0x0000000175bb8000,0x0000000175fbb000)] + 0x00000008d4cc7600 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=49923, stack(0x0000000175fc4000,0x00000001763c7000)] + 0x00000008d4ccde00 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50179, stack(0x00000001763d0000,0x00000001767d3000)] + 0x00000008d4d00000 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58883, stack(0x00000001767dc000,0x0000000176bdf000)] + 0x00000008d4d00600 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=58371, stack(0x0000000176be8000,0x0000000176feb000)] + 0x00000008d4d01800 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58115, stack(0x0000000176ff4000,0x00000001773f7000)] + 0x00000008d4d01e00 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=57603, stack(0x0000000177400000,0x0000000177803000)] + 0x00000008d4d02a00 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=50947, stack(0x000000017780c000,0x0000000177c0f000)] + 0x00000008d4d03000 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57091, stack(0x0000000177c18000,0x000000017801b000)] + 0x00000008d4d03600 JavaThread "task-abort-timer" daemon [_thread_blocked, id=51203, stack(0x0000000178024000,0x0000000178427000)] + 0x00000008d4d14600 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=51459, stack(0x0000000178430000,0x0000000178833000)] + 0x00000008d4d14c00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=51715, stack(0x000000017883c000,0x0000000178c3f000)] + 0x00000008d4d15e00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=56067, stack(0x0000000178c48000,0x000000017904b000)] + 0x00000008d4d16400 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=55555, stack(0x0000000179054000,0x0000000179457000)] + 0x00000008d4d17000 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=55043, stack(0x0000000179460000,0x0000000179863000)] + 0x00000008d4d28000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54787, stack(0x000000017986c000,0x0000000179c6f000)] + 0x00000008d4d28600 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=52739, stack(0x0000000179c78000,0x000000017a07b000)] + 0x00000008d4d28c00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=53251, stack(0x000000017a084000,0x000000017a487000)] + 0x00000008d4d29200 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=53507, stack(0x000000017a490000,0x000000017a893000)] + 0x00000008d4d69800 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=54019, stack(0x000000017a89c000,0x000000017ac9f000)] + 0x00000008d4d6b000 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=65539, stack(0x000000017aca8000,0x000000017b0ab000)] + 0x00000008d4d94000 JavaThread "Timer-0" [_thread_blocked, id=66051, stack(0x000000017b0b4000,0x000000017b4b7000)] + 0x00000008d4d96a00 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=86787, stack(0x000000017b4c0000,0x000000017b8c3000)] + 0x00000008d4b7c000 JavaThread "process reaper" daemon [_thread_blocked, id=66563, stack(0x000000017bce4000,0x000000017bd1b000)] + 0x00000008d513d800 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=41779, stack(0x000000017b8cc000,0x000000017bccf000)] + 0x00000008d513de00 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=24851, stack(0x000000017bd24000,0x000000017c127000)] +=>0x00000008d513e400 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=86283, stack(0x000000017c130000,0x000000017c533000)] + 0x00000008d513ea00 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=85775, stack(0x000000017c53c000,0x000000017c93f000)] + 0x00000008d513f000 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=85507, stack(0x000000017c948000,0x000000017cd4b000)] + 0x00000008d5147600 JavaThread "process reaper" daemon [_thread_blocked, id=83971, stack(0x000000016c8b0000,0x000000016c8e7000)] + 0x00000008ce8e2a00 JavaThread "process reaper" daemon [_thread_blocked, id=83203, stack(0x000000016c8f0000,0x000000016c927000)] + 0x00000008d68d5800 JavaThread "process reaper" daemon [_thread_blocked, id=82691, stack(0x000000017d778000,0x000000017d7af000)] + 0x00000008cfaf4c00 JavaThread "process reaper" daemon [_thread_blocked, id=82179, stack(0x000000017d7b8000,0x000000017d7ef000)] + 0x00000008d4bfea00 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=42535, stack(0x000000017cd54000,0x000000017d157000)] + 0x00000008d4bfde00 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=67343, stack(0x000000017d160000,0x000000017d563000)] + 0x00000008cc179200 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=81971, stack(0x000000017d7f8000,0x000000017dbfb000)] + 0x00000008d5144000 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=68891, stack(0x000000017dc04000,0x000000017e007000)] + 0x00000008d708ea00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69155, stack(0x000000017e010000,0x000000017e413000)] + 0x00000008d708f000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=81731, stack(0x000000017e41c000,0x000000017e81f000)] + 0x00000008d708f600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=69379, stack(0x000000017e828000,0x000000017ec2b000)] + 0x00000008d70e4000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81155, stack(0x000000017ec34000,0x000000017f037000)] + 0x00000008d70e4600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=69635, stack(0x000000017f040000,0x000000017f443000)] + 0x00000008d4bfc600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=78095, stack(0x000000016e190000,0x000000016e393000)] + 0x00000008cc043600 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=79123, stack(0x000000016e39c000,0x000000016e59f000)] + 0x00000008cc179800 JavaThread "C2 CompilerThread2" daemon [_thread_in_native, id=78599, stack(0x000000016e5a8000,0x000000016e7ab000)] + 0x00000008cc178000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=43059, stack(0x000000016e7b4000,0x000000016e9b7000)] + 0x00000008d11a4c00 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=64051, stack(0x000000016e9c0000,0x000000016ebc3000)] + 0x00000008d70e7000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=45351, stack(0x00000004115d8000,0x00000004119db000)] + +Other Threads: + 0x000000010568da50 VMThread "VM Thread" [stack: 0x000000016be00000,0x000000016c003000] [id=20227] + 0x00000008cc07d200 WatcherThread [stack: 0x000000016f1e4000,0x000000016f3e7000] [id=34051] + 0x0000000105686440 GCTaskThread "GC Thread#0" [stack: 0x000000016b3c4000,0x000000016b5c7000] [id=14595] + 0x00000008cea71b00 GCTaskThread "GC Thread#1" [stack: 0x000000016f5fc000,0x000000016f7ff000] [id=34563] + 0x00000008cea71e00 GCTaskThread "GC Thread#2" [stack: 0x000000016f808000,0x000000016fa0b000] [id=41475] + 0x00000008cea72100 GCTaskThread "GC Thread#3" [stack: 0x000000016fa14000,0x000000016fc17000] [id=41219] + 0x00000008cea72400 GCTaskThread "GC Thread#4" [stack: 0x000000016fc20000,0x000000016fe23000] [id=35331] + 0x00000008cea72700 GCTaskThread "GC Thread#5" [stack: 0x000000016fe2c000,0x000000017002f000] [id=40707] + 0x00000008cea72a00 GCTaskThread "GC Thread#6" [stack: 0x0000000170038000,0x000000017023b000] [id=40195] + 0x00000008cea72d00 GCTaskThread "GC Thread#7" [stack: 0x0000000170244000,0x0000000170447000] [id=39939] + 0x00000008cea73000 GCTaskThread "GC Thread#8" [stack: 0x0000000170450000,0x0000000170653000] [id=36355] + 0x00000008cea73300 GCTaskThread "GC Thread#9" [stack: 0x000000017065c000,0x000000017085f000] [id=39427] + 0x00000008cea73600 GCTaskThread "GC Thread#10" [stack: 0x0000000170868000,0x0000000170a6b000] [id=36867] + 0x00000008cea73900 GCTaskThread "GC Thread#11" [stack: 0x0000000170a74000,0x0000000170c77000] [id=39171] + 0x00000008cea73c00 GCTaskThread "GC Thread#12" [stack: 0x0000000170c80000,0x0000000170e83000] [id=38659] + 0x00000008cff9c000 GCTaskThread "GC Thread#13" [stack: 0x0000000170e8c000,0x000000017108f000] [id=37379] + 0x00000008cff9c300 GCTaskThread "GC Thread#14" [stack: 0x0000000171098000,0x000000017129b000] [id=37635] + 0x00000008cff9c600 GCTaskThread "GC Thread#15" [stack: 0x00000001712a4000,0x00000001714a7000] [id=43523] + 0x00000008cff9c900 GCTaskThread "GC Thread#16" [stack: 0x00000001714b0000,0x00000001716b3000] [id=65027] + 0x00000008cff9cc00 GCTaskThread "GC Thread#17" [stack: 0x00000001716bc000,0x00000001718bf000] [id=64771] + 0x00000008cff9cf00 GCTaskThread "GC Thread#18" [stack: 0x00000001718c8000,0x0000000171acb000] [id=44291] + 0x00000008cff9d200 GCTaskThread "GC Thread#19" [stack: 0x0000000171ad4000,0x0000000171cd7000] [id=44547] + 0x0000000105686d40 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016b5d0000,0x000000016b7d3000] [id=14339] + 0x0000000105687640 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016b7dc000,0x000000016b9df000] [id=14083] + 0x00000008d47c8c00 ConcurrentGCThread "G1 Conc#1" [stack: 0x00000001722f8000,0x00000001724fb000] [id=45571] + 0x00000008d47c9200 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000172504000,0x0000000172707000] [id=63747] + 0x00000008d47c9500 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000172710000,0x0000000172913000] [id=46339] + 0x00000008d47c9800 ConcurrentGCThread "G1 Conc#4" [stack: 0x000000017291c000,0x0000000172b1f000] [id=63491] + 0x00000001056897e0 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016b9e8000,0x000000016bbeb000] [id=16643] + 0x000000010568a0e0 ConcurrentGCThread "G1 Service" [stack: 0x000000016bbf4000,0x000000016bdf7000] [id=21507] + +Threads with active compile tasks: +C2 CompilerThread2 20989 14625 4 java.io.ObjectInputStream$FieldValues:: (217 bytes) + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x0000000300800000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x0000000500000000-0x0000000500bc0000-0x0000000500bc0000), size 12320768, SharedBaseAddress: 0x0000000500000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x0000000501000000-0x0000000541000000, reserved size: 1073741824 +Narrow klass base: 0x0000000500000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 342016K, used 246166K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 51 young (104448K), 5 survivors (10240K) + Metaspace used 151361K, committed 152640K, reserved 1245184K + class space used 18196K, committed 18752K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000300800000, 0x0000000300a00000, 0x0000000300a00000|100%|HS| |TAMS 0x0000000300a00000, 0x0000000300800000| Complete +| 1|0x0000000300a00000, 0x0000000300c00000, 0x0000000300c00000|100%|HS| |TAMS 0x0000000300c00000, 0x0000000300a00000| Complete +| 2|0x0000000300c00000, 0x0000000300e00000, 0x0000000300e00000|100%|HC| |TAMS 0x0000000300e00000, 0x0000000300c00000| Complete +| 3|0x0000000300e00000, 0x0000000301000000, 0x0000000301000000|100%|HS| |TAMS 0x0000000301000000, 0x0000000300e00000| Complete +| 4|0x0000000301000000, 0x0000000301200000, 0x0000000301200000|100%|HC| |TAMS 0x0000000301200000, 0x0000000301000000| Complete +| 5|0x0000000301200000, 0x0000000301400000, 0x0000000301400000|100%|HC| |TAMS 0x0000000301400000, 0x0000000301200000| Complete +| 6|0x0000000301400000, 0x0000000301600000, 0x0000000301600000|100%|HC| |TAMS 0x0000000301600000, 0x0000000301400000| Complete +| 7|0x0000000301600000, 0x0000000301800000, 0x0000000301800000|100%|HC| |TAMS 0x0000000301800000, 0x0000000301600000| Complete +| 8|0x0000000301800000, 0x0000000301a00000, 0x0000000301a00000|100%|HC| |TAMS 0x0000000301a00000, 0x0000000301800000| Complete +| 9|0x0000000301a00000, 0x0000000301c00000, 0x0000000301c00000|100%|HC| |TAMS 0x0000000301c00000, 0x0000000301a00000| Complete +| 10|0x0000000301c00000, 0x0000000301e00000, 0x0000000301e00000|100%|HC| |TAMS 0x0000000301e00000, 0x0000000301c00000| Complete +| 11|0x0000000301e00000, 0x0000000302000000, 0x0000000302000000|100%|HS| |TAMS 0x0000000302000000, 0x0000000301e00000| Complete +| 12|0x0000000302000000, 0x0000000302200000, 0x0000000302200000|100%|HS| |TAMS 0x0000000302200000, 0x0000000302000000| Complete +| 13|0x0000000302200000, 0x0000000302400000, 0x0000000302400000|100%|HC| |TAMS 0x0000000302400000, 0x0000000302200000| Complete +| 14|0x0000000302400000, 0x0000000302600000, 0x0000000302600000|100%|HS| |TAMS 0x0000000302600000, 0x0000000302400000| Complete +| 15|0x0000000302600000, 0x0000000302800000, 0x0000000302800000|100%|HC| |TAMS 0x0000000302800000, 0x0000000302600000| Complete +| 16|0x0000000302800000, 0x0000000302a00000, 0x0000000302a00000|100%|HS| |TAMS 0x0000000302a00000, 0x0000000302800000| Complete +| 17|0x0000000302a00000, 0x0000000302c00000, 0x0000000302c00000|100%|HS| |TAMS 0x0000000302c00000, 0x0000000302a00000| Complete +| 18|0x0000000302c00000, 0x0000000302e00000, 0x0000000302e00000|100%|HC| |TAMS 0x0000000302e00000, 0x0000000302c00000| Complete +| 19|0x0000000302e00000, 0x0000000303000000, 0x0000000303000000|100%| O| |TAMS 0x0000000303000000, 0x0000000302e00000| Untracked +| 20|0x0000000303000000, 0x0000000303200000, 0x0000000303200000|100%| O| |TAMS 0x0000000303200000, 0x0000000303000000| Untracked +| 21|0x0000000303200000, 0x0000000303400000, 0x0000000303400000|100%| O| |TAMS 0x0000000303400000, 0x0000000303200000| Untracked +| 22|0x0000000303400000, 0x0000000303600000, 0x0000000303600000|100%| O| |TAMS 0x0000000303600000, 0x0000000303400000| Untracked +| 23|0x0000000303600000, 0x0000000303800000, 0x0000000303800000|100%| O| |TAMS 0x0000000303800000, 0x0000000303600000| Untracked +| 24|0x0000000303800000, 0x0000000303a00000, 0x0000000303a00000|100%| O| |TAMS 0x0000000303a00000, 0x0000000303800000| Untracked +| 25|0x0000000303a00000, 0x0000000303c00000, 0x0000000303c00000|100%| O| |TAMS 0x0000000303c00000, 0x0000000303a00000| Untracked +| 26|0x0000000303c00000, 0x0000000303e00000, 0x0000000303e00000|100%| O| |TAMS 0x0000000303e00000, 0x0000000303c00000| Untracked +| 27|0x0000000303e00000, 0x0000000304000000, 0x0000000304000000|100%| O| |TAMS 0x0000000304000000, 0x0000000303e00000| Untracked +| 28|0x0000000304000000, 0x0000000304200000, 0x0000000304200000|100%| O| |TAMS 0x0000000304200000, 0x0000000304000000| Untracked +| 29|0x0000000304200000, 0x0000000304400000, 0x0000000304400000|100%| O| |TAMS 0x0000000304400000, 0x0000000304200000| Untracked +| 30|0x0000000304400000, 0x0000000304600000, 0x0000000304600000|100%| O| |TAMS 0x0000000304600000, 0x0000000304400000| Untracked +| 31|0x0000000304600000, 0x0000000304800000, 0x0000000304800000|100%| O| |TAMS 0x0000000304800000, 0x0000000304600000| Untracked +| 32|0x0000000304800000, 0x0000000304a00000, 0x0000000304a00000|100%| O| |TAMS 0x0000000304a00000, 0x0000000304800000| Untracked +| 33|0x0000000304a00000, 0x0000000304c00000, 0x0000000304c00000|100%| O| |TAMS 0x0000000304c00000, 0x0000000304a00000| Untracked +| 34|0x0000000304c00000, 0x0000000304e00000, 0x0000000304e00000|100%| O| |TAMS 0x0000000304e00000, 0x0000000304c00000| Untracked +| 35|0x0000000304e00000, 0x0000000305000000, 0x0000000305000000|100%| O| |TAMS 0x0000000305000000, 0x0000000304e00000| Untracked +| 36|0x0000000305000000, 0x0000000305200000, 0x0000000305200000|100%| O| |TAMS 0x0000000305200000, 0x0000000305000000| Untracked +| 37|0x0000000305200000, 0x0000000305400000, 0x0000000305400000|100%| O| |TAMS 0x0000000305400000, 0x0000000305200000| Untracked +| 38|0x0000000305400000, 0x0000000305600000, 0x0000000305600000|100%| O| |TAMS 0x0000000305600000, 0x0000000305400000| Untracked +| 39|0x0000000305600000, 0x0000000305800000, 0x0000000305800000|100%| O| |TAMS 0x0000000305800000, 0x0000000305600000| Untracked +| 40|0x0000000305800000, 0x0000000305a00000, 0x0000000305a00000|100%| O| |TAMS 0x0000000305a00000, 0x0000000305800000| Untracked +| 41|0x0000000305a00000, 0x0000000305bbee00, 0x0000000305c00000| 87%| O| |TAMS 0x0000000305bbee00, 0x0000000305a00000| Untracked +| 42|0x0000000305c00000, 0x0000000305e00000, 0x0000000305e00000|100%|HS| |TAMS 0x0000000305c00000, 0x0000000305c00000| Complete +| 43|0x0000000305e00000, 0x0000000306000000, 0x0000000306000000|100%|HS| |TAMS 0x0000000305e00000, 0x0000000305e00000| Complete +| 44|0x0000000306000000, 0x0000000306200000, 0x0000000306200000|100%|HS| |TAMS 0x0000000306000000, 0x0000000306000000| Complete +| 45|0x0000000306200000, 0x0000000306400000, 0x0000000306400000|100%|HS| |TAMS 0x0000000306200000, 0x0000000306200000| Complete +| 46|0x0000000306400000, 0x0000000306600000, 0x0000000306600000|100%|HS| |TAMS 0x0000000306400000, 0x0000000306400000| Complete +| 47|0x0000000306600000, 0x0000000306800000, 0x0000000306800000|100%|HS| |TAMS 0x0000000306600000, 0x0000000306600000| Complete +| 48|0x0000000306800000, 0x0000000306a00000, 0x0000000306a00000|100%| O| |TAMS 0x0000000306800000, 0x0000000306800000| Untracked +| 49|0x0000000306a00000, 0x0000000306c00000, 0x0000000306c00000|100%| O| |TAMS 0x0000000306a00000, 0x0000000306a00000| Untracked +| 50|0x0000000306c00000, 0x0000000306e00000, 0x0000000306e00000|100%|HS| |TAMS 0x0000000306c00000, 0x0000000306c00000| Complete +| 51|0x0000000306e00000, 0x0000000307000000, 0x0000000307000000|100%|HS| |TAMS 0x0000000306e00000, 0x0000000306e00000| Complete +| 52|0x0000000307000000, 0x0000000307200000, 0x0000000307200000|100%|HS| |TAMS 0x0000000307000000, 0x0000000307000000| Complete +| 53|0x0000000307200000, 0x0000000307400000, 0x0000000307400000|100%|HS| |TAMS 0x0000000307200000, 0x0000000307200000| Complete +| 54|0x0000000307400000, 0x0000000307600000, 0x0000000307600000|100%| O| |TAMS 0x0000000307400000, 0x0000000307400000| Untracked +| 55|0x0000000307600000, 0x0000000307800000, 0x0000000307800000|100%| O| |TAMS 0x0000000307600000, 0x0000000307600000| Untracked +| 56|0x0000000307800000, 0x0000000307a00000, 0x0000000307a00000|100%| O| |TAMS 0x0000000307800000, 0x0000000307800000| Untracked +| 57|0x0000000307a00000, 0x0000000307c00000, 0x0000000307c00000|100%| O| |TAMS 0x0000000307a00000, 0x0000000307a00000| Untracked +| 58|0x0000000307c00000, 0x0000000307e00000, 0x0000000307e00000|100%|HS| |TAMS 0x0000000307c00000, 0x0000000307c00000| Complete +| 59|0x0000000307e00000, 0x0000000308000000, 0x0000000308000000|100%|HC| |TAMS 0x0000000307e00000, 0x0000000307e00000| Complete +| 60|0x0000000308000000, 0x0000000308200000, 0x0000000308200000|100%|HC| |TAMS 0x0000000308000000, 0x0000000308000000| Complete +| 61|0x0000000308200000, 0x0000000308200000, 0x0000000308400000| 0%| F| |TAMS 0x0000000308200000, 0x0000000308200000| Untracked +| 62|0x0000000308400000, 0x0000000308400000, 0x0000000308600000| 0%| F| |TAMS 0x0000000308400000, 0x0000000308400000| Untracked +| 63|0x0000000308600000, 0x0000000308600000, 0x0000000308800000| 0%| F| |TAMS 0x0000000308600000, 0x0000000308600000| Untracked +| 64|0x0000000308800000, 0x0000000308800000, 0x0000000308a00000| 0%| F| |TAMS 0x0000000308800000, 0x0000000308800000| Untracked +| 65|0x0000000308a00000, 0x0000000308a00000, 0x0000000308c00000| 0%| F| |TAMS 0x0000000308a00000, 0x0000000308a00000| Untracked +| 66|0x0000000308c00000, 0x0000000308c00000, 0x0000000308e00000| 0%| F| |TAMS 0x0000000308c00000, 0x0000000308c00000| Untracked +| 67|0x0000000308e00000, 0x0000000308e00000, 0x0000000309000000| 0%| F| |TAMS 0x0000000308e00000, 0x0000000308e00000| Untracked +| 68|0x0000000309000000, 0x0000000309000000, 0x0000000309200000| 0%| F| |TAMS 0x0000000309000000, 0x0000000309000000| Untracked +| 69|0x0000000309200000, 0x0000000309200000, 0x0000000309400000| 0%| F| |TAMS 0x0000000309200000, 0x0000000309200000| Untracked +| 70|0x0000000309400000, 0x000000030957ce98, 0x0000000309600000| 74%| S|CS|TAMS 0x0000000309400000, 0x0000000309400000| Complete +| 71|0x0000000309600000, 0x0000000309800000, 0x0000000309800000|100%| O| |TAMS 0x0000000309600000, 0x0000000309600000| Untracked +| 72|0x0000000309800000, 0x0000000309a00000, 0x0000000309a00000|100%| O| |TAMS 0x0000000309800000, 0x0000000309800000| Untracked +| 73|0x0000000309a00000, 0x0000000309c00000, 0x0000000309c00000|100%| O| |TAMS 0x0000000309a00000, 0x0000000309a00000| Untracked +| 74|0x0000000309c00000, 0x0000000309e00000, 0x0000000309e00000|100%| O| |TAMS 0x0000000309c00000, 0x0000000309c00000| Untracked +| 75|0x0000000309e00000, 0x000000030a000000, 0x000000030a000000|100%| O| |TAMS 0x0000000309e00000, 0x0000000309e00000| Untracked +| 76|0x000000030a000000, 0x000000030a200000, 0x000000030a200000|100%| O| |TAMS 0x000000030a000000, 0x000000030a000000| Untracked +| 77|0x000000030a200000, 0x000000030a400000, 0x000000030a400000|100%| O| |TAMS 0x000000030a200000, 0x000000030a200000| Untracked +| 78|0x000000030a400000, 0x000000030a600000, 0x000000030a600000|100%| O| |TAMS 0x000000030a400000, 0x000000030a400000| Untracked +| 79|0x000000030a600000, 0x000000030a741e00, 0x000000030a800000| 62%| O| |TAMS 0x000000030a600000, 0x000000030a600000| Untracked +| 80|0x000000030a800000, 0x000000030a800000, 0x000000030aa00000| 0%| F| |TAMS 0x000000030a800000, 0x000000030a800000| Untracked +| 81|0x000000030aa00000, 0x000000030aa00000, 0x000000030ac00000| 0%| F| |TAMS 0x000000030aa00000, 0x000000030aa00000| Untracked +| 82|0x000000030ac00000, 0x000000030ae00000, 0x000000030ae00000|100%| S|CS|TAMS 0x000000030ac00000, 0x000000030ac00000| Complete +| 83|0x000000030ae00000, 0x000000030ae00000, 0x000000030b000000| 0%| F| |TAMS 0x000000030ae00000, 0x000000030ae00000| Untracked +| 84|0x000000030b000000, 0x000000030b000000, 0x000000030b200000| 0%| F| |TAMS 0x000000030b000000, 0x000000030b000000| Untracked +| 85|0x000000030b200000, 0x000000030b400000, 0x000000030b400000|100%| S|CS|TAMS 0x000000030b200000, 0x000000030b200000| Complete +| 86|0x000000030b400000, 0x000000030b600000, 0x000000030b600000|100%| S|CS|TAMS 0x000000030b400000, 0x000000030b400000| Complete +| 87|0x000000030b600000, 0x000000030b800000, 0x000000030b800000|100%| S|CS|TAMS 0x000000030b600000, 0x000000030b600000| Complete +| 88|0x000000030b800000, 0x000000030b800000, 0x000000030ba00000| 0%| F| |TAMS 0x000000030b800000, 0x000000030b800000| Untracked +| 89|0x000000030ba00000, 0x000000030ba00000, 0x000000030bc00000| 0%| F| |TAMS 0x000000030ba00000, 0x000000030ba00000| Untracked +| 90|0x000000030bc00000, 0x000000030bc00000, 0x000000030be00000| 0%| F| |TAMS 0x000000030bc00000, 0x000000030bc00000| Untracked +| 91|0x000000030be00000, 0x000000030be00000, 0x000000030c000000| 0%| F| |TAMS 0x000000030be00000, 0x000000030be00000| Untracked +| 92|0x000000030c000000, 0x000000030c000000, 0x000000030c200000| 0%| F| |TAMS 0x000000030c000000, 0x000000030c000000| Untracked +| 93|0x000000030c200000, 0x000000030c200000, 0x000000030c400000| 0%| F| |TAMS 0x000000030c200000, 0x000000030c200000| Untracked +| 94|0x000000030c400000, 0x000000030c400000, 0x000000030c600000| 0%| F| |TAMS 0x000000030c400000, 0x000000030c400000| Untracked +| 95|0x000000030c600000, 0x000000030c600000, 0x000000030c800000| 0%| F| |TAMS 0x000000030c600000, 0x000000030c600000| Untracked +| 96|0x000000030c800000, 0x000000030c800000, 0x000000030ca00000| 0%| F| |TAMS 0x000000030c800000, 0x000000030c800000| Untracked +| 97|0x000000030ca00000, 0x000000030ca00000, 0x000000030cc00000| 0%| F| |TAMS 0x000000030ca00000, 0x000000030ca00000| Untracked +| 98|0x000000030cc00000, 0x000000030cc00000, 0x000000030ce00000| 0%| F| |TAMS 0x000000030cc00000, 0x000000030cc00000| Untracked +| 99|0x000000030ce00000, 0x000000030ce00000, 0x000000030d000000| 0%| F| |TAMS 0x000000030ce00000, 0x000000030ce00000| Untracked +| 100|0x000000030d000000, 0x000000030d000000, 0x000000030d200000| 0%| F| |TAMS 0x000000030d000000, 0x000000030d000000| Untracked +| 101|0x000000030d200000, 0x000000030d200000, 0x000000030d400000| 0%| F| |TAMS 0x000000030d200000, 0x000000030d200000| Untracked +| 102|0x000000030d400000, 0x000000030d400000, 0x000000030d600000| 0%| F| |TAMS 0x000000030d400000, 0x000000030d400000| Untracked +| 103|0x000000030d600000, 0x000000030d600000, 0x000000030d800000| 0%| F| |TAMS 0x000000030d600000, 0x000000030d600000| Untracked +| 104|0x000000030d800000, 0x000000030d800000, 0x000000030da00000| 0%| F| |TAMS 0x000000030d800000, 0x000000030d800000| Untracked +| 105|0x000000030da00000, 0x000000030da00000, 0x000000030dc00000| 0%| F| |TAMS 0x000000030da00000, 0x000000030da00000| Untracked +| 106|0x000000030dc00000, 0x000000030dc00000, 0x000000030de00000| 0%| F| |TAMS 0x000000030dc00000, 0x000000030dc00000| Untracked +| 107|0x000000030de00000, 0x000000030de00000, 0x000000030e000000| 0%| F| |TAMS 0x000000030de00000, 0x000000030de00000| Untracked +| 108|0x000000030e000000, 0x000000030e000000, 0x000000030e200000| 0%| F| |TAMS 0x000000030e000000, 0x000000030e000000| Untracked +| 109|0x000000030e200000, 0x000000030e200000, 0x000000030e400000| 0%| F| |TAMS 0x000000030e200000, 0x000000030e200000| Untracked +| 110|0x000000030e400000, 0x000000030e400000, 0x000000030e600000| 0%| F| |TAMS 0x000000030e400000, 0x000000030e400000| Untracked +| 111|0x000000030e600000, 0x000000030e600000, 0x000000030e800000| 0%| F| |TAMS 0x000000030e600000, 0x000000030e600000| Untracked +| 112|0x000000030e800000, 0x000000030e800000, 0x000000030ea00000| 0%| F| |TAMS 0x000000030e800000, 0x000000030e800000| Untracked +| 113|0x000000030ea00000, 0x000000030ea00000, 0x000000030ec00000| 0%| F| |TAMS 0x000000030ea00000, 0x000000030ea00000| Untracked +| 114|0x000000030ec00000, 0x000000030ec00000, 0x000000030ee00000| 0%| F| |TAMS 0x000000030ec00000, 0x000000030ec00000| Untracked +| 115|0x000000030ee00000, 0x000000030ee00000, 0x000000030f000000| 0%| F| |TAMS 0x000000030ee00000, 0x000000030ee00000| Untracked +| 116|0x000000030f000000, 0x000000030f000000, 0x000000030f200000| 0%| F| |TAMS 0x000000030f000000, 0x000000030f000000| Untracked +| 117|0x000000030f200000, 0x000000030f200000, 0x000000030f400000| 0%| F| |TAMS 0x000000030f200000, 0x000000030f200000| Untracked +| 118|0x000000030f400000, 0x000000030f400000, 0x000000030f600000| 0%| F| |TAMS 0x000000030f400000, 0x000000030f400000| Untracked +| 119|0x000000030f600000, 0x000000030f722fa0, 0x000000030f800000| 56%| E| |TAMS 0x000000030f600000, 0x000000030f600000| Complete +| 120|0x000000030f800000, 0x000000030fa00000, 0x000000030fa00000|100%| E|CS|TAMS 0x000000030f800000, 0x000000030f800000| Complete +| 121|0x000000030fa00000, 0x000000030fc00000, 0x000000030fc00000|100%| E|CS|TAMS 0x000000030fa00000, 0x000000030fa00000| Complete +| 122|0x000000030fc00000, 0x000000030fe00000, 0x000000030fe00000|100%| E|CS|TAMS 0x000000030fc00000, 0x000000030fc00000| Complete +| 123|0x000000030fe00000, 0x0000000310000000, 0x0000000310000000|100%| E|CS|TAMS 0x000000030fe00000, 0x000000030fe00000| Complete +| 124|0x0000000310000000, 0x0000000310200000, 0x0000000310200000|100%| E|CS|TAMS 0x0000000310000000, 0x0000000310000000| Complete +| 125|0x0000000310200000, 0x0000000310400000, 0x0000000310400000|100%| E|CS|TAMS 0x0000000310200000, 0x0000000310200000| Complete +| 126|0x0000000310400000, 0x0000000310600000, 0x0000000310600000|100%| E|CS|TAMS 0x0000000310400000, 0x0000000310400000| Complete +| 127|0x0000000310600000, 0x0000000310800000, 0x0000000310800000|100%| E|CS|TAMS 0x0000000310600000, 0x0000000310600000| Complete +| 128|0x0000000310800000, 0x0000000310a00000, 0x0000000310a00000|100%| E|CS|TAMS 0x0000000310800000, 0x0000000310800000| Complete +| 129|0x0000000310a00000, 0x0000000310c00000, 0x0000000310c00000|100%| E|CS|TAMS 0x0000000310a00000, 0x0000000310a00000| Complete +| 130|0x0000000310c00000, 0x0000000310e00000, 0x0000000310e00000|100%| E|CS|TAMS 0x0000000310c00000, 0x0000000310c00000| Complete +| 131|0x0000000310e00000, 0x0000000311000000, 0x0000000311000000|100%| E|CS|TAMS 0x0000000310e00000, 0x0000000310e00000| Complete +| 132|0x0000000311000000, 0x0000000311200000, 0x0000000311200000|100%| E|CS|TAMS 0x0000000311000000, 0x0000000311000000| Complete +| 133|0x0000000311200000, 0x0000000311400000, 0x0000000311400000|100%| E|CS|TAMS 0x0000000311200000, 0x0000000311200000| Complete +| 134|0x0000000311400000, 0x0000000311600000, 0x0000000311600000|100%| E|CS|TAMS 0x0000000311400000, 0x0000000311400000| Complete +| 135|0x0000000311600000, 0x0000000311800000, 0x0000000311800000|100%| E|CS|TAMS 0x0000000311600000, 0x0000000311600000| Complete +| 136|0x0000000311800000, 0x0000000311a00000, 0x0000000311a00000|100%| E|CS|TAMS 0x0000000311800000, 0x0000000311800000| Complete +| 137|0x0000000311a00000, 0x0000000311c00000, 0x0000000311c00000|100%| E|CS|TAMS 0x0000000311a00000, 0x0000000311a00000| Complete +| 138|0x0000000311c00000, 0x0000000311e00000, 0x0000000311e00000|100%| E|CS|TAMS 0x0000000311c00000, 0x0000000311c00000| Complete +| 139|0x0000000311e00000, 0x0000000312000000, 0x0000000312000000|100%| E|CS|TAMS 0x0000000311e00000, 0x0000000311e00000| Complete +| 140|0x0000000312000000, 0x0000000312200000, 0x0000000312200000|100%| E|CS|TAMS 0x0000000312000000, 0x0000000312000000| Complete +| 141|0x0000000312200000, 0x0000000312400000, 0x0000000312400000|100%| E|CS|TAMS 0x0000000312200000, 0x0000000312200000| Complete +| 142|0x0000000312400000, 0x0000000312600000, 0x0000000312600000|100%| E|CS|TAMS 0x0000000312400000, 0x0000000312400000| Complete +| 143|0x0000000312600000, 0x0000000312800000, 0x0000000312800000|100%| E|CS|TAMS 0x0000000312600000, 0x0000000312600000| Complete +| 144|0x0000000312800000, 0x0000000312a00000, 0x0000000312a00000|100%| E|CS|TAMS 0x0000000312800000, 0x0000000312800000| Complete +| 145|0x0000000312a00000, 0x0000000312c00000, 0x0000000312c00000|100%| E|CS|TAMS 0x0000000312a00000, 0x0000000312a00000| Complete +| 146|0x0000000312c00000, 0x0000000312e00000, 0x0000000312e00000|100%| E|CS|TAMS 0x0000000312c00000, 0x0000000312c00000| Complete +| 147|0x0000000312e00000, 0x0000000313000000, 0x0000000313000000|100%| E|CS|TAMS 0x0000000312e00000, 0x0000000312e00000| Complete +| 148|0x0000000313000000, 0x0000000313200000, 0x0000000313200000|100%| E|CS|TAMS 0x0000000313000000, 0x0000000313000000| Complete +| 149|0x0000000313200000, 0x0000000313400000, 0x0000000313400000|100%| E|CS|TAMS 0x0000000313200000, 0x0000000313200000| Complete +| 150|0x0000000313400000, 0x0000000313600000, 0x0000000313600000|100%| E|CS|TAMS 0x0000000313400000, 0x0000000313400000| Complete +| 151|0x0000000313600000, 0x0000000313800000, 0x0000000313800000|100%| E|CS|TAMS 0x0000000313600000, 0x0000000313600000| Complete +| 152|0x0000000313800000, 0x0000000313a00000, 0x0000000313a00000|100%| E|CS|TAMS 0x0000000313800000, 0x0000000313800000| Complete +| 153|0x0000000313a00000, 0x0000000313c00000, 0x0000000313c00000|100%| E|CS|TAMS 0x0000000313a00000, 0x0000000313a00000| Complete +| 154|0x0000000313c00000, 0x0000000313e00000, 0x0000000313e00000|100%| E|CS|TAMS 0x0000000313c00000, 0x0000000313c00000| Complete +| 155|0x0000000313e00000, 0x0000000314000000, 0x0000000314000000|100%| E|CS|TAMS 0x0000000313e00000, 0x0000000313e00000| Complete +| 156|0x0000000314000000, 0x0000000314200000, 0x0000000314200000|100%| E|CS|TAMS 0x0000000314000000, 0x0000000314000000| Complete +| 157|0x0000000314200000, 0x0000000314400000, 0x0000000314400000|100%| E|CS|TAMS 0x0000000314200000, 0x0000000314200000| Complete +| 158|0x0000000314400000, 0x0000000314600000, 0x0000000314600000|100%| E|CS|TAMS 0x0000000314400000, 0x0000000314400000| Complete +| 340|0x000000032b000000, 0x000000032b200000, 0x000000032b200000|100%| E|CS|TAMS 0x000000032b000000, 0x000000032b000000| Complete +| 341|0x000000032b200000, 0x000000032b400000, 0x000000032b400000|100%| E|CS|TAMS 0x000000032b200000, 0x000000032b200000| Complete +| 440|0x0000000337800000, 0x0000000337a00000, 0x0000000337a00000|100%| E|CS|TAMS 0x0000000337800000, 0x0000000337800000| Complete +| 441|0x0000000337a00000, 0x0000000337c00000, 0x0000000337c00000|100%| E|CS|TAMS 0x0000000337a00000, 0x0000000337a00000| Complete +| 442|0x0000000337c00000, 0x0000000337e00000, 0x0000000337e00000|100%| E|CS|TAMS 0x0000000337c00000, 0x0000000337c00000| Complete +| 767|0x0000000360600000, 0x0000000360800000, 0x0000000360800000|100%| E|CS|TAMS 0x0000000360600000, 0x0000000360600000| Complete +|2046|0x0000000400400000, 0x0000000400574000, 0x0000000400600000| 72%|OA| |TAMS 0x0000000400574000, 0x0000000400400000| Untracked +|2047|0x0000000400600000, 0x0000000400674000, 0x0000000400800000| 22%|CA| |TAMS 0x0000000400674000, 0x0000000400600000| Untracked + +Card table byte_map: [0x00000001225b8000,0x0000000122db8000] _byte_map_base: 0x0000000120db4000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x00000008c9094010, (CMBitMap*) 0x00000008c9094050 + Prev Bits: [0x00000001235b8000, 0x00000001275b8000) + Next Bits: [0x00000001275b8000, 0x000000012b5b8000) + +Polling page: 0x0000000104efc000 + +Metaspace: + +Usage: + Non-class: 130.04 MB used. + Class: 17.77 MB used. + Both: 147.81 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 130.75 MB ( 68%) committed, 3 nodes. + Class space: 1.00 GB reserved, 18.31 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 149.06 MB ( 12%) committed. + +Chunk freelists: + Non-Class: 13.06 MB + Class: 13.70 MB + Both: 26.76 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 164.81 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 14. +num_arena_births: 2064. +num_arena_deaths: 0. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2382. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 14. +num_chunks_taken_from_freelist: 7764. +num_chunk_merges: 11. +num_chunk_splits: 5785. +num_chunks_enlarged: 4384. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=9399Kb max_used=9399Kb free=109769Kb + bounds [0x000000011b158000, 0x000000011ba88000, 0x00000001225b8000] +CodeHeap 'profiled nmethods': size=119152Kb used=19479Kb max_used=19791Kb free=99672Kb + bounds [0x00000001135b8000, 0x0000000114918000, 0x000000011aa14000] +CodeHeap 'non-nmethods': size=7440Kb used=3323Kb max_used=3484Kb free=4116Kb + bounds [0x000000011aa14000, 0x000000011ad94000, 0x000000011b158000] + total_blobs=11894 nmethods=10833 adapters=973 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 20.875 Thread 0x00000008cc043600 14786 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) +Event: 20.875 Thread 0x00000008cc178000 14787 3 com.google.protobuf.MapEntry::writeTo (26 bytes) +Event: 20.875 Thread 0x00000008cc043600 nmethod 14786 0x000000011489fa90 code [0x000000011489fc80, 0x000000011489fee8] +Event: 20.875 Thread 0x00000008cc178000 nmethod 14787 0x000000011443a110 code [0x000000011443a2c0, 0x000000011443a4b8] +Event: 20.875 Thread 0x00000008cc042400 nmethod 14781 0x0000000114439190 code [0x0000000114439440, 0x0000000114439cf8] +Event: 20.875 Thread 0x00000008cc042400 14790 3 org.apache.comet.ConfigEntry::key (13 bytes) +Event: 20.875 Thread 0x00000008cc043600 14789 3 scala.collection.convert.Wrappers$IteratorWrapper::$jacocoInit (52 bytes) +Event: 20.875 Thread 0x00000008cc178000 14791 3 scala.collection.immutable.Map$EmptyMap$::updated (21 bytes) +Event: 20.875 Thread 0x00000008cc042400 nmethod 14790 0x0000000114438d10 code [0x0000000114438ec0, 0x0000000114439048] +Event: 20.875 Thread 0x00000008cc042400 14792 3 java.util.GregorianCalendar::computeFields (85 bytes) +Event: 20.875 Thread 0x00000008d11a4c00 nmethod 14782 0x0000000113a75490 code [0x0000000113a75800, 0x0000000113a76b58] +Event: 20.875 Thread 0x00000008cc178000 nmethod 14791 0x0000000114438790 code [0x0000000114438940, 0x0000000114438b78] +Event: 20.875 Thread 0x00000008cc178000 14788 3 scala.collection.immutable.List::apply (15 bytes) +Event: 20.875 Thread 0x00000008cc178000 nmethod 14788 0x0000000113a74f90 code [0x0000000113a75140, 0x0000000113a75338] +Event: 20.875 Thread 0x00000008d11a4c00 14793 3 org.apache.comet.serde.Config$ConfigMap$Builder::internalGetMutableEntries (110 bytes) +Event: 20.875 Thread 0x00000008cc178000 14794 3 org.apache.comet.serde.Config$ConfigMap$Builder::putEntries (87 bytes) +Event: 20.875 Thread 0x00000008cc042400 nmethod 14792 0x0000000113a74790 code [0x0000000113a749c0, 0x0000000113a74e18] +Event: 20.875 Thread 0x00000008cc043600 nmethod 14789 0x0000000113a73910 code [0x0000000113a73b80, 0x0000000113a74538] +Event: 20.875 Thread 0x00000008d11a4c00 nmethod 14793 0x0000000113a72f90 code [0x0000000113a731c0, 0x0000000113a73758] +Event: 20.875 Thread 0x00000008cc178000 nmethod 14794 0x0000000113a72510 code [0x0000000113a72740, 0x0000000113a72cc8] + +GC Heap History (20 events): +Event: 6.523 GC heap before +{Heap before GC invocations=25 (full 0): + garbage-first heap total 342016K, used 283317K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 85 young (174080K), 13 survivors (26624K) + Metaspace used 123782K, committed 124608K, reserved 1179648K + class space used 14869K, committed 15296K, reserved 1048576K +} +Event: 6.526 GC heap after +{Heap after GC invocations=26 (full 0): + garbage-first heap total 342016K, used 148707K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 6 young (12288K), 6 survivors (12288K) + Metaspace used 123782K, committed 124608K, reserved 1179648K + class space used 14869K, committed 15296K, reserved 1048576K +} +Event: 6.818 GC heap before +{Heap before GC invocations=26 (full 0): + garbage-first heap total 342016K, used 277731K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 69 young (141312K), 6 survivors (12288K) + Metaspace used 127657K, committed 128512K, reserved 1179648K + class space used 15121K, committed 15488K, reserved 1048576K +} +Event: 6.819 GC heap after +{Heap after GC invocations=27 (full 0): + garbage-first heap total 342016K, used 147405K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 127657K, committed 128512K, reserved 1179648K + class space used 15121K, committed 15488K, reserved 1048576K +} +Event: 7.010 GC heap before +{Heap before GC invocations=27 (full 0): + garbage-first heap total 342016K, used 290765K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 70 young (143360K), 5 survivors (10240K) + Metaspace used 133183K, committed 134080K, reserved 1179648K + class space used 15911K, committed 16320K, reserved 1048576K +} +Event: 7.011 GC heap after +{Heap after GC invocations=28 (full 0): + garbage-first heap total 342016K, used 133772K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 133183K, committed 134080K, reserved 1179648K + class space used 15911K, committed 16320K, reserved 1048576K +} +Event: 7.271 GC heap before +{Heap before GC invocations=28 (full 0): + garbage-first heap total 342016K, used 291468K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 76 young (155648K), 3 survivors (6144K) + Metaspace used 140639K, committed 141632K, reserved 1179648K + class space used 16853K, committed 17280K, reserved 1048576K +} +Event: 7.273 GC heap after +{Heap after GC invocations=29 (full 0): + garbage-first heap total 342016K, used 135270K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 140639K, committed 141632K, reserved 1179648K + class space used 16853K, committed 17280K, reserved 1048576K +} +Event: 7.490 GC heap before +{Heap before GC invocations=29 (full 0): + garbage-first heap total 342016K, used 288870K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 77 young (157696K), 4 survivors (8192K) + Metaspace used 148294K, committed 149376K, reserved 1245184K + class space used 17691K, committed 18176K, reserved 1048576K +} +Event: 7.492 GC heap after +{Heap after GC invocations=30 (full 0): + garbage-first heap total 342016K, used 137462K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148294K, committed 149376K, reserved 1245184K + class space used 17691K, committed 18176K, reserved 1048576K +} +Event: 7.850 GC heap before +{Heap before GC invocations=30 (full 0): + garbage-first heap total 342016K, used 286966K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 78 young (159744K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 7.853 GC heap after +{Heap after GC invocations=31 (full 0): + garbage-first heap total 342016K, used 137868K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.168 GC heap before +{Heap before GC invocations=31 (full 0): + garbage-first heap total 342016K, used 291468K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 80 young (163840K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.170 GC heap after +{Heap after GC invocations=32 (full 0): + garbage-first heap total 342016K, used 137599K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.482 GC heap before +{Heap before GC invocations=32 (full 0): + garbage-first heap total 342016K, used 293247K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 81 young (165888K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.484 GC heap after +{Heap after GC invocations=33 (full 0): + garbage-first heap total 342016K, used 137844K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.800 GC heap before +{Heap before GC invocations=33 (full 0): + garbage-first heap total 342016K, used 293492K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 81 young (165888K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 8.802 GC heap after +{Heap after GC invocations=34 (full 0): + garbage-first heap total 342016K, used 137466K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 9.102 GC heap before +{Heap before GC invocations=34 (full 0): + garbage-first heap total 342016K, used 293114K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 81 young (165888K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} +Event: 9.104 GC heap after +{Heap after GC invocations=35 (full 0): + garbage-first heap total 342016K, used 137622K [0x0000000300800000, 0x0000000400800000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 148795K, committed 149888K, reserved 1245184K + class space used 17760K, committed 18240K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.397 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li +Event: 3.397 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.397 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.362 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-16636361560025204640.dylib +Event: 5.872 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 5.872 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 5.872 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 5.872 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 5.872 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 5.872 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 5.872 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.872 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 5.873 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 5.873 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 6.021 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15419921919615002278.dylib +Event: 6.735 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-cd884805-d66d-4abd-b6de-6004019a551b-libsnappyjava.dylib + +Deoptimization events (20 events): +Event: 20.871 Thread 0x00000008d513de00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 +Event: 20.871 Thread 0x00000008d513f000 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 +Event: 20.871 Thread 0x00000008d513e400 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 +Event: 20.871 Thread 0x00000008d513ea00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 +Event: 20.871 Thread 0x00000008d513e400 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 +Event: 20.871 Thread 0x00000008d513e400 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c5312e0 +Event: 20.871 Thread 0x00000008d513de00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 +Event: 20.871 Thread 0x00000008d513e400 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c531290 mode 2 +Event: 20.871 Thread 0x00000008d513de00 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c1252e0 +Event: 20.871 Thread 0x00000008d513de00 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c125290 mode 2 +Event: 20.871 Thread 0x00000008d513d800 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 +Event: 20.871 Thread 0x00000008d513f000 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 +Event: 20.871 Thread 0x00000008d513f000 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017cd492e0 +Event: 20.871 Thread 0x00000008d513d800 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 +Event: 20.871 Thread 0x00000008d513ea00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 +Event: 20.871 Thread 0x00000008d513d800 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017bccd2e0 +Event: 20.871 Thread 0x00000008d513f000 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017cd49290 mode 2 +Event: 20.871 Thread 0x00000008d513ea00 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c93d2e0 +Event: 20.871 Thread 0x00000008d513d800 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017bccd290 mode 2 +Event: 20.871 Thread 0x00000008d513ea00 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c93d290 mode 2 + +Classes loaded (20 events): +Event: 7.424 Loading class java/lang/Class$EnclosingMethodInfo +Event: 7.424 Loading class java/lang/Class$EnclosingMethodInfo done +Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper +Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$Unmapper +Event: 15.931 Loading class jdk/internal/access/foreign/UnmapperProxy +Event: 15.931 Loading class jdk/internal/access/foreign/UnmapperProxy done +Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$Unmapper done +Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done +Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask +Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask +Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done +Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done +Event: 20.497 Loading class java/util/function/LongBinaryOperator +Event: 20.497 Loading class java/util/function/LongBinaryOperator done +Event: 20.725 Loading class sun/nio/ch/ChannelInputStream +Event: 20.725 Loading class sun/nio/ch/ChannelInputStream done +Event: 20.810 Loading class java/math/BigDecimal$StringBuilderHelper +Event: 20.810 Loading class java/math/BigDecimal$StringBuilderHelper done +Event: 20.871 Loading class java/nio/channels/Channels$ReadableByteChannelImpl +Event: 20.872 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 20.867 Thread 0x00000008d513f000 Exception (0x000000030fd89a68) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513de00 Exception (0x000000030fc51c08) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513ea00 Exception (0x0000000310280458) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513d800 Exception (0x000000030fa2d460) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513e400 Exception (0x000000030fdcf718) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513f000 Exception (0x000000030fd8d080) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.867 Thread 0x00000008d513de00 Exception (0x000000030fc55220) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb62968) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fda1250) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513ea00 Exception (0x00000003102955a0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513d800 Exception (0x000000030fa7a188) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb66958) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513e400 Exception (0x000000030fddfed0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fda5028) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513ea00 Exception (0x0000000310299378) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb6a238) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fbb2798) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513ea00 Exception (0x000000030f802738) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513d800 Exception (0x000000030fa7dbe8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.869 Thread 0x00000008d513e400 Exception (0x000000030fde3930) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] + +VM Operations (20 events): +Event: 7.573 Executing VM operation: ICBufferFull +Event: 7.573 Executing VM operation: ICBufferFull done +Event: 7.850 Executing VM operation: G1CollectForAllocation +Event: 7.853 Executing VM operation: G1CollectForAllocation done +Event: 8.168 Executing VM operation: G1CollectForAllocation +Event: 8.170 Executing VM operation: G1CollectForAllocation done +Event: 8.482 Executing VM operation: G1CollectForAllocation +Event: 8.484 Executing VM operation: G1CollectForAllocation done +Event: 8.800 Executing VM operation: G1CollectForAllocation +Event: 8.802 Executing VM operation: G1CollectForAllocation done +Event: 9.102 Executing VM operation: G1CollectForAllocation +Event: 9.104 Executing VM operation: G1CollectForAllocation done +Event: 12.119 Executing VM operation: Cleanup +Event: 12.119 Executing VM operation: Cleanup done +Event: 16.139 Executing VM operation: Cleanup +Event: 16.139 Executing VM operation: Cleanup done +Event: 20.799 Executing VM operation: ICBufferFull +Event: 20.799 Executing VM operation: ICBufferFull done +Event: 20.833 Executing VM operation: ICBufferFull +Event: 20.833 Executing VM operation: ICBufferFull done + +Memory protections (20 events): +Event: 7.055 Protecting memory [0x000000017e010000,0x000000017e01c000] with protection modes 3 +Event: 7.279 Protecting memory [0x000000017e010000,0x000000017e01c000] with protection modes 0 +Event: 7.281 Protecting memory [0x000000017e41c000,0x000000017e428000] with protection modes 0 +Event: 7.281 Protecting memory [0x000000017e828000,0x000000017e834000] with protection modes 0 +Event: 7.281 Protecting memory [0x000000017ec34000,0x000000017ec40000] with protection modes 0 +Event: 7.281 Protecting memory [0x000000017f040000,0x000000017f04c000] with protection modes 0 +Event: 9.190 Protecting memory [0x00000001720ec000,0x00000001720f8000] with protection modes 3 +Event: 9.190 Protecting memory [0x000000016e39c000,0x000000016e3a8000] with protection modes 3 +Event: 9.190 Protecting memory [0x000000016f3f0000,0x000000016f3fc000] with protection modes 3 +Event: 9.192 Protecting memory [0x000000016e190000,0x000000016e19c000] with protection modes 3 +Event: 9.192 Protecting memory [0x000000016ebcc000,0x000000016ebd8000] with protection modes 3 +Event: 9.203 Protecting memory [0x000000016e9c0000,0x000000016e9cc000] with protection modes 3 +Event: 9.203 Protecting memory [0x000000016e7b4000,0x000000016e7c0000] with protection modes 3 +Event: 14.208 Protecting memory [0x000000016e5a8000,0x000000016e5b4000] with protection modes 3 +Event: 20.727 Protecting memory [0x000000016e190000,0x000000016e19c000] with protection modes 0 +Event: 20.747 Protecting memory [0x000000016e39c000,0x000000016e3a8000] with protection modes 0 +Event: 20.747 Protecting memory [0x000000016e5a8000,0x000000016e5b4000] with protection modes 0 +Event: 20.747 Protecting memory [0x000000016e7b4000,0x000000016e7c0000] with protection modes 0 +Event: 20.748 Protecting memory [0x000000016e9c0000,0x000000016e9cc000] with protection modes 0 +Event: 20.801 Protecting memory [0x00000004115d8000,0x00000004115e4000] with protection modes 0 + +Nmethod flushes (20 events): +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114889d90 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011488ac10 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114894790 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114894c10 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114895a90 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114896190 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114897790 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114898610 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114899490 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489a210 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489ad10 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489d490 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489fa90 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489ff10 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a3c10 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a4090 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a4610 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a6790 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a6c90 +Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a7a90 + +Events (20 events): +Event: 7.055 Thread 0x00000008d11a4600 Thread exited: 0x00000008d11a4600 +Event: 7.279 Thread 0x00000008d4d00000 Thread added: 0x00000008d708ea00 +Event: 7.281 Thread 0x00000008d708ea00 Thread added: 0x00000008d708f000 +Event: 7.281 Thread 0x00000008d708ea00 Thread added: 0x00000008d708f600 +Event: 7.281 Thread 0x00000008d708f600 Thread added: 0x00000008d70e4000 +Event: 7.281 Thread 0x00000008d70e4000 Thread added: 0x00000008d70e4600 +Event: 9.190 Thread 0x00000008d513f600 Thread exited: 0x00000008d513f600 +Event: 9.190 Thread 0x00000008d11a4c00 Thread exited: 0x00000008d11a4c00 +Event: 9.190 Thread 0x00000008cc178000 Thread exited: 0x00000008cc178000 +Event: 9.192 Thread 0x00000008cc043600 Thread exited: 0x00000008cc043600 +Event: 9.192 Thread 0x00000008d4bfc600 Thread exited: 0x00000008d4bfc600 +Event: 9.203 Thread 0x00000008d4bfc000 Thread exited: 0x00000008d4bfc000 +Event: 9.203 Thread 0x00000008cc179800 Thread exited: 0x00000008cc179800 +Event: 14.208 Thread 0x00000008d68d7600 Thread exited: 0x00000008d68d7600 +Event: 20.727 Thread 0x00000008cc042400 Thread added: 0x00000008d4bfc600 +Event: 20.747 Thread 0x00000008cc042400 Thread added: 0x00000008cc043600 +Event: 20.747 Thread 0x00000008cc043600 Thread added: 0x00000008cc179800 +Event: 20.747 Thread 0x00000008cc043600 Thread added: 0x00000008cc178000 +Event: 20.748 Thread 0x00000008cc043600 Thread added: 0x00000008d11a4c00 +Event: 20.801 Thread 0x00000008d4d14600 Thread added: 0x00000008d70e7000 + + +Dynamic libraries: +0x0000000104f14000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x000000010aa74000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x0000000104f3c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x0000000104f6c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x0000000104fbc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x000000010a478000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x000000010513c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x000000010a4c4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x000000010a4a0000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x000000010a568000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x000000010a57c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x000000010a590000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x000000010a5a4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000400800000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-16636361560025204640.dylib +0x000000010a67c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15419921919615002278.dylib +0x000000010a650000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-cd884805-d66d-4abd-b6de-6004019a551b-libsnappyjava.dylib + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:39 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 10.03 7.52 5.24 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(1442992k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/hs_err_pid59058.log b/spark/hs_err_pid59058.log new file mode 100644 index 0000000000..8df2ae515e --- /dev/null +++ b/spark/hs_err_pid59058.log @@ -0,0 +1,1568 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x000000010a7c1530, pid=59058, tid=63499 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 08:58:03 2026 MDT elapsed time: 20.974203 seconds (0d 0h 0m 20s) + +--------------- T H R E A D --------------- + +Current thread (0x0000000b661e1200): JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=63499, stack(0x000000017c1b0000,0x000000017c5b3000)] + +Stack: [0x000000017c1b0000,0x000000017c5b3000], sp=0x000000017c5aac00, free space=4075k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 +C [libcomet-14395648754114117926.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 +C [libcomet-14395648754114117926.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc +C [libcomet-14395648754114117926.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 +C [libcomet-14395648754114117926.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 +C [libcomet-14395648754114117926.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c +C [libcomet-14395648754114117926.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 +C [libcomet-14395648754114117926.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 +C [libcomet-14395648754114117926.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 +C [libcomet-14395648754114117926.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 +C [libcomet-14395648754114117926.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 +C [libcomet-14395648754114117926.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 +C [libcomet-14395648754114117926.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 +C [libcomet-14395648754114117926.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c +C [libcomet-14395648754114117926.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 +C [libcomet-14395648754114117926.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 +C [libcomet-14395648754114117926.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c +C [libcomet-14395648754114117926.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 +C [libcomet-14395648754114117926.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 +C [libcomet-14395648754114117926.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 +C [libcomet-14395648754114117926.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c +C [libcomet-14395648754114117926.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 +C [libcomet-14395648754114117926.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 +C [libcomet-14395648754114117926.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 +C [libcomet-14395648754114117926.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c +C [libcomet-14395648754114117926.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c +C [libcomet-14395648754114117926.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-14395648754114117926.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-14395648754114117926.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-14395648754114117926.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-14395648754114117926.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-14395648754114117926.dylib+0x106628] __rust_try+0x20 +C [libcomet-14395648754114117926.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-14395648754114117926.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-14395648754114117926.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-14395648754114117926.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3971+0x000000f8021c5bd8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3969+0x000000f8021c4da8.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2516+0x000000f801df1670.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3971+0x000000f8021c5bd8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3969+0x000000f8021c4da8.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2516+0x000000f801df1670.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 + +Registers: + x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000b64545270 + x4=0x000000017c5aad20 x5=0x0000000b661e1200 x6=0x0000000b7450a600 x7=0xfffff0003ffff800 + x8=0x000000f8021ade48 x9=0x0000000000000000 x10=0x000000f800000000 x11=0x0000000000000004 +x12=0x0000000139051af8 x13=0x0000000000000000 x14=0xfffffffffffdb1ac x15=0x0000000b66c48000 +x16=0x0000000000000040 x17=0x0000000b67100af0 x18=0x0000000000000000 x19=0x0000000b661e1200 +x20=0x000000017c5aad50 x21=0x000000017c5aad20 x22=0x0000000b6540ff18 x23=0x0000000713a9de98 +x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000b65c56f00 x27=0x0000000702bb6b20 +x28=0x0000000000000001 fp=0x000000017c5aad10 lr=0x000000010a7c152c sp=0x000000017c5aac00 +pc=0x000000010a7c1530 cpsr=0x0000000000001000 + +Register to memory mapping: + + x0=0x0 is NULL + x1=0x0000000000000007 is an unknown value + x2=0x0000000000000001 is an unknown value + x3=0x0000000b64545270 points into unknown readable memory: 0x0000000139051af8 | f8 1a 05 39 01 00 00 00 + x4=0x000000017c5aad20 is pointing into the stack for thread: 0x0000000b661e1200 + x5=0x0000000b661e1200 is a thread + x6=0x0000000b7450a600 points into unknown readable memory: 0x0000000b66578900 | 00 89 57 66 0b 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=0x000000f8021ade48 is pointing into metadata + x9=0x0 is NULL +x10=0x000000f800000000 is pointing into metadata +x11=0x0000000000000004 is an unknown value +x12={method} {0x0000000139051af8} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' +x13=0x0 is NULL +x14=0xfffffffffffdb1ac is an unknown value +x15=0x0000000b66c48000 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +x16=0x0000000000000040 is an unknown value +x17=0x0000000b67100af0 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff +x18=0x0 is NULL +x19=0x0000000b661e1200 is a thread +x20=0x000000017c5aad50 is pointing into the stack for thread: 0x0000000b661e1200 +x21=0x000000017c5aad20 is pointing into the stack for thread: 0x0000000b661e1200 +x22=0x0000000b6540ff18 points into unknown readable memory: 0x0000000713a9de98 | 98 de a9 13 07 00 00 00 +x23=0x0000000713a9de98 is an oop: org.apache.comet.CometHandleBatchIterator +{0x0000000713a9de98} - klass: 'org/apache/comet/CometHandleBatchIterator' + - ---- fields (total size 2 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x0000000713f31818} (e27e6303) +x24=0x0000000000000001 is an unknown value +x25=0x0 is NULL +x26=0x0000000b65c56f00 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x27=0x0000000702bb6b20 is an oop: [Z +{0x0000000702bb6b20} - klass: {type array bool} + - length: 9 +x28=0x0000000000000001 is an unknown value + + +Top of Stack: (sp=0x000000017c5aac00) +0x000000017c5aac00: 000000017c5a52c0 0000000000000001 +0x000000017c5aac10: 0000000b747601e0 0000000153212428 +0x000000017c5aac20: 0000000b71b3eac0 0000000b6c79d380 +0x000000017c5aac30: 000000017c5aac67 0000000000000000 +0x000000017c5aac40: 000000017c5aac80 00000001507800d0 +0x000000017c5aac50: 0000000000000081 0000000b664b5ed8 +0x000000017c5aac60: 0000000b74760320 0000000b664b5ed8 +0x000000017c5aac70: 0000000b664b5ed8 ff00ffffffffffff +0x000000017c5aac80: 000000017c5aad10 00000001507aff58 +0x000000017c5aac90: 0000000000000063 000000017c5aad20 +0x000000017c5aaca0: 000000017c5aad10 000000010a7ddf24 +0x000000017c5aacb0: 5bfd0d834fab00d0 0000000b661e1200 +0x000000017c5aacc0: 0000000b661e1200 0000000702bb6b20 +0x000000017c5aacd0: 0000000138fd9018 0000000000000000 +0x000000017c5aace0: 0000000b661e14b0 0000000000000008 +0x000000017c5aacf0: 0000000b668e81e0 0000000b64545270 +0x000000017c5aad00: 0000000000000001 0000000b661e1200 +0x000000017c5aad10: 000000017c5aadc0 000000010a7c560c +0x000000017c5aad20: 000000010af066b8 000000f80045a1d0 +0x000000017c5aad30: 000000010000000a 0000000000000014 +0x000000017c5aad40: 0000000000000000 0000000000000008 +0x000000017c5aad50: 000000000000000a 000000010af6ff90 +0x000000017c5aad60: 0000000b661e1200 0000000000000000 +0x000000017c5aad70: 000000017c5aadc0 0000000149e84c58 +0x000000017c5aad80: 0000000138fd9018 0000000702b3d680 +0x000000017c5aad90: 000000017c5b1818 0000000702b3d078 +0x000000017c5aada0: 0000000000000000 000000010af6ff90 +0x000000017c5aadb0: 000000017c5b1738 000000011a2ab72c +0x000000017c5aadc0: 000000017c5ac980 0000000149e49560 +0x000000017c5aadd0: 0000000000000000 0000000000000081 +0x000000017c5aade0: 0000000000000082 0000000b74760320 +0x000000017c5aadf0: 0000000000000082 ffff00ff00ff00ff + +Instructions: (pc=0x000000010a7c1530) +0x000000010a7c1430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 +0x000000010a7c1440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 +0x000000010a7c1450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 +0x000000010a7c1460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 +0x000000010a7c1470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 +0x000000010a7c1480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 +0x000000010a7c1490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 +0x000000010a7c14a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 +0x000000010a7c14b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa +0x000000010a7c14c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 +0x000000010a7c14d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 +0x000000010a7c14e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 +0x000000010a7c14f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a +0x000000010a7c1500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa +0x000000010a7c1510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 +0x000000010a7c1520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa +0x000000010a7c1530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 +0x000000010a7c1540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 +0x000000010a7c1550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 +0x000000010a7c1560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 +0x000000010a7c1570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 +0x000000010a7c1580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 +0x000000010a7c1590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa +0x000000010a7c15a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 +0x000000010a7c15b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 +0x000000010a7c15c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 +0x000000010a7c15d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 +0x000000010a7c15e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 +0x000000010a7c15f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa +0x000000010a7c1600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 +0x000000010a7c1610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 +0x000000010a7c1620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x000000017c5a52c0 is pointing into the stack for thread: 0x0000000b661e1200 +stack at sp + 1 slots: 0x0000000000000001 is an unknown value +stack at sp + 2 slots: 0x0000000b747601e0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +stack at sp + 3 slots: 0x0000000153212428: _ZN5comet7parquet15parquet_support15PARQUET_OPTIONS17h20af737816b13564E+0x28650 in /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib at 0x0000000149dd0000 +stack at sp + 4 slots: 0x0000000b71b3eac0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +stack at sp + 5 slots: 0x0000000b6c79d380 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +stack at sp + 6 slots: 0x000000017c5aac67 is pointing into the stack for thread: 0x0000000b661e1200 +stack at sp + 7 slots: 0x0 is NULL + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x0000000b63d63900, length=77, elements={ +0x0000000104f0c330, 0x0000000104f1a620, 0x0000000b668e4000, 0x0000000b668e4c00, +0x0000000b668e5200, 0x0000000b668e5800, 0x0000000b668e5e00, 0x0000000b668e6400, +0x0000000b668e6a00, 0x0000000b668e7000, 0x0000000b66932400, 0x0000000b66b74600, +0x0000000b66725e00, 0x0000000b65437000, 0x0000000b65434000, 0x0000000b66b0f000, +0x0000000b66b76a00, 0x0000000b66b75200, 0x0000000b65005e00, 0x0000000b65006400, +0x0000000b65006a00, 0x0000000b65007000, 0x0000000b65007600, 0x0000000b65014000, +0x0000000b65014600, 0x0000000b65014c00, 0x0000000b65017000, 0x0000000b65529200, +0x0000000b65529800, 0x0000000b6552aa00, 0x0000000b6552b000, 0x0000000b65094000, +0x0000000b65094600, 0x0000000b65094c00, 0x0000000b65095800, 0x0000000b65095e00, +0x0000000b65097000, 0x0000000b65097600, 0x0000000b650b4600, 0x0000000b650b5200, +0x0000000b650b5800, 0x0000000b650b5e00, 0x0000000b650b6400, 0x0000000b65136a00, +0x0000000b65178600, 0x0000000b65179200, 0x0000000b6517b600, 0x0000000b64fe7000, +0x0000000b661e1200, 0x0000000b661e1800, 0x0000000b661e1e00, 0x0000000b661e2400, +0x0000000b661e2a00, 0x0000000b6679b000, 0x0000000b65ca0000, 0x0000000b65ca0c00, +0x0000000b65ca1800, 0x0000000b6619b600, 0x0000000b65663600, 0x0000000b660ac000, +0x0000000b6517b000, 0x0000000b64460000, 0x0000000b64460600, 0x0000000b64460c00, +0x0000000b64461200, 0x0000000b64461800, 0x0000000b6451f600, 0x0000000b668e7600, +0x0000000b65684000, 0x0000000b66930000, 0x0000000b66b0e400, 0x0000000b66198000, +0x0000000b661e3600, 0x0000000b6451c600, 0x0000000b64463600, 0x0000000b63d64000, +0x0000000b63d64600 +} + +Java Threads: ( => current thread ) + 0x0000000104f0c330 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5635, stack(0x000000016b690000,0x000000016ba93000)] + 0x0000000104f1a620 JavaThread "Reference Handler" daemon [_thread_blocked, id=23299, stack(0x000000016c6e4000,0x000000016cae7000)] + 0x0000000b668e4000 JavaThread "Finalizer" daemon [_thread_blocked, id=31491, stack(0x000000016caf0000,0x000000016cef3000)] + 0x0000000b668e4c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=29699, stack(0x000000016d014000,0x000000016d417000)] + 0x0000000b668e5200 JavaThread "Service Thread" daemon [_thread_blocked, id=29187, stack(0x000000016d420000,0x000000016d823000)] + 0x0000000b668e5800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=28675, stack(0x000000016d82c000,0x000000016dc2f000)] + 0x0000000b668e5e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=25347, stack(0x000000016dc38000,0x000000016de3b000)] + 0x0000000b668e6400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27907, stack(0x000000016de44000,0x000000016e047000)] + 0x0000000b668e6a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27395, stack(0x000000016e050000,0x000000016e453000)] + 0x0000000b668e7000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=25603, stack(0x000000016e45c000,0x000000016e85f000)] + 0x0000000b66932400 JavaThread "Notification Thread" daemon [_thread_blocked, id=42499, stack(0x000000016f2a4000,0x000000016f6a7000)] + 0x0000000b66b74600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45059, stack(0x00000001725c4000,0x00000001729c7000)] + 0x0000000b66725e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=62211, stack(0x000000017340c000,0x000000017380f000)] + 0x0000000b65437000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=46851, stack(0x0000000173818000,0x0000000173c1b000)] + 0x0000000b65434000 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=47107, stack(0x0000000173c24000,0x0000000174027000)] + 0x0000000b66b0f000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=61443, stack(0x0000000174030000,0x0000000174433000)] + 0x0000000b66b76a00 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=47619, stack(0x000000017443c000,0x000000017483f000)] + 0x0000000b66b75200 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=60931, stack(0x0000000174848000,0x0000000174c4b000)] + 0x0000000b65005e00 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=60675, stack(0x0000000174c54000,0x0000000175057000)] + 0x0000000b65006400 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=60419, stack(0x0000000175060000,0x0000000175463000)] + 0x0000000b65006a00 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=60163, stack(0x000000017546c000,0x000000017586f000)] + 0x0000000b65007000 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=48643, stack(0x0000000175878000,0x0000000175c7b000)] + 0x0000000b65007600 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=49155, stack(0x0000000175c84000,0x0000000176087000)] + 0x0000000b65014000 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=49667, stack(0x0000000176090000,0x0000000176493000)] + 0x0000000b65014600 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49923, stack(0x000000017649c000,0x000000017689f000)] + 0x0000000b65014c00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=50179, stack(0x00000001768a8000,0x0000000176cab000)] + 0x0000000b65017000 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=59139, stack(0x0000000176cb4000,0x00000001770b7000)] + 0x0000000b65529200 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58883, stack(0x00000001770c0000,0x00000001774c3000)] + 0x0000000b65529800 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=51203, stack(0x00000001774cc000,0x00000001778cf000)] + 0x0000000b6552aa00 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58627, stack(0x00000001778d8000,0x0000000177cdb000)] + 0x0000000b6552b000 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=58371, stack(0x0000000177ce4000,0x00000001780e7000)] + 0x0000000b65094000 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=51971, stack(0x00000001780f0000,0x00000001784f3000)] + 0x0000000b65094600 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57859, stack(0x00000001784fc000,0x00000001788ff000)] + 0x0000000b65094c00 JavaThread "task-abort-timer" daemon [_thread_blocked, id=52483, stack(0x0000000178908000,0x0000000178d0b000)] + 0x0000000b65095800 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=57091, stack(0x0000000178d14000,0x0000000179117000)] + 0x0000000b65095e00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52739, stack(0x0000000179120000,0x0000000179523000)] + 0x0000000b65097000 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=56323, stack(0x000000017952c000,0x000000017992f000)] + 0x0000000b65097600 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=53251, stack(0x0000000179938000,0x0000000179d3b000)] + 0x0000000b650b4600 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=53763, stack(0x0000000179d44000,0x000000017a147000)] + 0x0000000b650b5200 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54019, stack(0x000000017a150000,0x000000017a553000)] + 0x0000000b650b5800 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54531, stack(0x000000017a55c000,0x000000017a95f000)] + 0x0000000b650b5e00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=54787, stack(0x000000017a968000,0x000000017ad6b000)] + 0x0000000b650b6400 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55299, stack(0x000000017ad74000,0x000000017b177000)] + 0x0000000b65136a00 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=65539, stack(0x000000017b180000,0x000000017b583000)] + 0x0000000b65178600 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=86787, stack(0x000000017b58c000,0x000000017b98f000)] + 0x0000000b65179200 JavaThread "Timer-0" [_thread_blocked, id=86275, stack(0x000000017b998000,0x000000017bd9b000)] + 0x0000000b6517b600 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=85763, stack(0x000000017bda4000,0x000000017c1a7000)] + 0x0000000b64fe7000 JavaThread "process reaper" daemon [_thread_blocked, id=34575, stack(0x000000016fcd4000,0x000000016fd0b000)] +=>0x0000000b661e1200 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=63499, stack(0x000000017c1b0000,0x000000017c5b3000)] + 0x0000000b661e1800 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=41739, stack(0x000000017c5bc000,0x000000017c9bf000)] + 0x0000000b661e1e00 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=24339, stack(0x000000017c9c8000,0x000000017cdcb000)] + 0x0000000b661e2400 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=65795, stack(0x000000017cdd4000,0x000000017d1d7000)] + 0x0000000b661e2a00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=66307, stack(0x000000017d1e0000,0x000000017d5e3000)] + 0x0000000b6679b000 JavaThread "process reaper" daemon [_thread_blocked, id=84483, stack(0x000000016cf88000,0x000000016cfbf000)] + 0x0000000b65ca0000 JavaThread "process reaper" daemon [_thread_blocked, id=67331, stack(0x000000016cfc8000,0x000000016cfff000)] + 0x0000000b65ca0c00 JavaThread "process reaper" daemon [_thread_blocked, id=68099, stack(0x000000016fd14000,0x000000016fd4b000)] + 0x0000000b65ca1800 JavaThread "process reaper" daemon [_thread_blocked, id=82947, stack(0x000000016fd54000,0x000000016fd8b000)] + 0x0000000b6619b600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=34063, stack(0x000000017d5ec000,0x000000017d9ef000)] + 0x0000000b65663600 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=68623, stack(0x000000017d9f8000,0x000000017ddfb000)] + 0x0000000b660ac000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=83507, stack(0x000000017de04000,0x000000017e207000)] + 0x0000000b6517b000 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=83219, stack(0x000000017e210000,0x000000017e613000)] + 0x0000000b64460000 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69179, stack(0x000000017e61c000,0x000000017ea1f000)] + 0x0000000b64460600 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=82747, stack(0x000000017ea28000,0x000000017ee2b000)] + 0x0000000b64460c00 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=82179, stack(0x000000017ee34000,0x000000017f237000)] + 0x0000000b64461200 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=69891, stack(0x000000017f240000,0x000000017f643000)] + 0x0000000b64461800 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=81923, stack(0x000000017f64c000,0x000000017fa4f000)] + 0x0000000b6451f600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=79879, stack(0x000000016e868000,0x000000016ea6b000)] + 0x0000000b668e7600 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=72207, stack(0x000000016ea74000,0x000000016ec77000)] + 0x0000000b65684000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=71947, stack(0x000000016ec80000,0x000000016ee83000)] + 0x0000000b66930000 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=25871, stack(0x000000016ee8c000,0x000000016f08f000)] + 0x0000000b66b0e400 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=43331, stack(0x000000016f098000,0x000000016f29b000)] + 0x0000000b66198000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=33311, stack(0x000000016f8bc000,0x000000016fcbf000)] + 0x0000000b661e3600 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=42787, stack(0x000000030041c000,0x000000030081f000)] + 0x0000000b6451c600 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=26635, stack(0x0000000300828000,0x0000000300c2b000)] + 0x0000000b64463600 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=26379, stack(0x0000000300e58000,0x000000030125b000)] + 0x0000000b63d64000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=84747, stack(0x0000000301264000,0x0000000301667000)] + 0x0000000b63d64600 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=79363, stack(0x0000000301670000,0x0000000301a73000)] + +Other Threads: + 0x0000000104f199a0 VMThread "VM Thread" [stack: 0x000000016c4d8000,0x000000016c6db000] [id=19715] + 0x0000000b66980600 WatcherThread [stack: 0x000000016f6b0000,0x000000016f8b3000] [id=33539] + 0x0000000104f13370 GCTaskThread "GC Thread#0" [stack: 0x000000016ba9c000,0x000000016bc9f000] [id=14083] + 0x0000000b66524f00 GCTaskThread "GC Thread#1" [stack: 0x000000016fee0000,0x00000001700e3000] [id=35075] + 0x0000000b66525200 GCTaskThread "GC Thread#2" [stack: 0x00000001700ec000,0x00000001702ef000] [id=35587] + 0x0000000b66525500 GCTaskThread "GC Thread#3" [stack: 0x00000001702f8000,0x00000001704fb000] [id=41219] + 0x0000000b66525800 GCTaskThread "GC Thread#4" [stack: 0x0000000170504000,0x0000000170707000] [id=40707] + 0x0000000b66525b00 GCTaskThread "GC Thread#5" [stack: 0x0000000170710000,0x0000000170913000] [id=36099] + 0x0000000b66525e00 GCTaskThread "GC Thread#6" [stack: 0x000000017091c000,0x0000000170b1f000] [id=40451] + 0x0000000b66526100 GCTaskThread "GC Thread#7" [stack: 0x0000000170b28000,0x0000000170d2b000] [id=40195] + 0x0000000b66526400 GCTaskThread "GC Thread#8" [stack: 0x0000000170d34000,0x0000000170f37000] [id=39683] + 0x0000000b66526700 GCTaskThread "GC Thread#9" [stack: 0x0000000170f40000,0x0000000171143000] [id=37123] + 0x0000000b66526a00 GCTaskThread "GC Thread#10" [stack: 0x000000017114c000,0x000000017134f000] [id=39427] + 0x0000000b66526d00 GCTaskThread "GC Thread#11" [stack: 0x0000000171358000,0x000000017155b000] [id=37635] + 0x0000000b66527000 GCTaskThread "GC Thread#12" [stack: 0x0000000171564000,0x0000000171767000] [id=37891] + 0x0000000b66527300 GCTaskThread "GC Thread#13" [stack: 0x0000000171770000,0x0000000171973000] [id=38403] + 0x0000000b66527600 GCTaskThread "GC Thread#14" [stack: 0x000000017197c000,0x0000000171b7f000] [id=43523] + 0x0000000b66527900 GCTaskThread "GC Thread#15" [stack: 0x0000000171b88000,0x0000000171d8b000] [id=65283] + 0x0000000b66527c00 GCTaskThread "GC Thread#16" [stack: 0x0000000171d94000,0x0000000171f97000] [id=44035] + 0x0000000b66b08000 GCTaskThread "GC Thread#17" [stack: 0x0000000171fa0000,0x00000001721a3000] [id=64771] + 0x0000000b66b08300 GCTaskThread "GC Thread#18" [stack: 0x00000001721ac000,0x00000001723af000] [id=64259] + 0x0000000b66b08600 GCTaskThread "GC Thread#19" [stack: 0x00000001723b8000,0x00000001725bb000] [id=44803] + 0x0000000104f13f70 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016bca8000,0x000000016beab000] [id=13827] + 0x0000000104f146e0 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016beb4000,0x000000016c0b7000] [id=13315] + 0x0000000b65430f00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000172bdc000,0x0000000172ddf000] [id=63235] + 0x0000000b65431500 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000172de8000,0x0000000172feb000] [id=45571] + 0x0000000b65431800 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000172ff4000,0x00000001731f7000] [id=46083] + 0x0000000b65431b00 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000173200000,0x0000000173403000] [id=62467] + 0x0000000104f16590 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016c0c0000,0x000000016c2c3000] [id=21507] + 0x0000000104f16c90 ConcurrentGCThread "G1 Service" [stack: 0x000000016c2cc000,0x000000016c4cf000] [id=16643] + +Threads with active compile tasks: + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x0000000700000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x000000f800000000-0x000000f800bc0000-0x000000f800bc0000), size 12320768, SharedBaseAddress: 0x000000f800000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x000000f801000000-0x000000f841000000, reserved size: 1073741824 +Narrow klass base: 0x000000f800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 348160K, used 181662K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 15 young (30720K), 4 survivors (8192K) + Metaspace used 151503K, committed 152768K, reserved 1245184K + class space used 18223K, committed 18816K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000700000000, 0x0000000700200000, 0x0000000700200000|100%|HS| |TAMS 0x0000000700200000, 0x0000000700200000| Complete +| 1|0x0000000700200000, 0x0000000700400000, 0x0000000700400000|100%|HS| |TAMS 0x0000000700400000, 0x0000000700400000| Complete +| 2|0x0000000700400000, 0x0000000700600000, 0x0000000700600000|100%|HC| |TAMS 0x0000000700600000, 0x0000000700600000| Complete +| 3|0x0000000700600000, 0x0000000700800000, 0x0000000700800000|100%|HS| |TAMS 0x0000000700800000, 0x0000000700800000| Complete +| 4|0x0000000700800000, 0x0000000700a00000, 0x0000000700a00000|100%|HC| |TAMS 0x0000000700a00000, 0x0000000700a00000| Complete +| 5|0x0000000700a00000, 0x0000000700c00000, 0x0000000700c00000|100%|HC| |TAMS 0x0000000700c00000, 0x0000000700c00000| Complete +| 6|0x0000000700c00000, 0x0000000700e00000, 0x0000000700e00000|100%|HC| |TAMS 0x0000000700e00000, 0x0000000700e00000| Complete +| 7|0x0000000700e00000, 0x0000000701000000, 0x0000000701000000|100%|HC| |TAMS 0x0000000701000000, 0x0000000701000000| Complete +| 8|0x0000000701000000, 0x0000000701200000, 0x0000000701200000|100%|HC| |TAMS 0x0000000701200000, 0x0000000701200000| Complete +| 9|0x0000000701200000, 0x0000000701400000, 0x0000000701400000|100%|HC| |TAMS 0x0000000701400000, 0x0000000701400000| Complete +| 10|0x0000000701400000, 0x0000000701600000, 0x0000000701600000|100%|HC| |TAMS 0x0000000701600000, 0x0000000701600000| Complete +| 11|0x0000000701600000, 0x0000000701800000, 0x0000000701800000|100%|HS| |TAMS 0x0000000701800000, 0x0000000701800000| Complete +| 12|0x0000000701800000, 0x0000000701a00000, 0x0000000701a00000|100%|HS| |TAMS 0x0000000701a00000, 0x0000000701a00000| Complete +| 13|0x0000000701a00000, 0x0000000701c00000, 0x0000000701c00000|100%|HC| |TAMS 0x0000000701c00000, 0x0000000701c00000| Complete +| 14|0x0000000701c00000, 0x0000000701e00000, 0x0000000701e00000|100%|HS| |TAMS 0x0000000701e00000, 0x0000000701e00000| Complete +| 15|0x0000000701e00000, 0x0000000702000000, 0x0000000702000000|100%|HC| |TAMS 0x0000000702000000, 0x0000000702000000| Complete +| 16|0x0000000702000000, 0x0000000702200000, 0x0000000702200000|100%|HS| |TAMS 0x0000000702200000, 0x0000000702200000| Complete +| 17|0x0000000702200000, 0x0000000702400000, 0x0000000702400000|100%|HS| |TAMS 0x0000000702400000, 0x0000000702400000| Complete +| 18|0x0000000702400000, 0x0000000702600000, 0x0000000702600000|100%|HC| |TAMS 0x0000000702600000, 0x0000000702600000| Complete +| 19|0x0000000702600000, 0x0000000702800000, 0x0000000702800000|100%| O| |TAMS 0x0000000702800000, 0x0000000702800000| Untracked +| 20|0x0000000702800000, 0x0000000702a00000, 0x0000000702a00000|100%| O| |TAMS 0x0000000702a00000, 0x0000000702a00000| Untracked +| 21|0x0000000702a00000, 0x0000000702c00000, 0x0000000702c00000|100%| O| |TAMS 0x0000000702c00000, 0x0000000702c00000| Untracked +| 22|0x0000000702c00000, 0x0000000702e00000, 0x0000000702e00000|100%| O| |TAMS 0x0000000702e00000, 0x0000000702e00000| Untracked +| 23|0x0000000702e00000, 0x0000000703000000, 0x0000000703000000|100%| O| |TAMS 0x0000000703000000, 0x0000000703000000| Untracked +| 24|0x0000000703000000, 0x0000000703200000, 0x0000000703200000|100%| O| |TAMS 0x0000000703200000, 0x0000000703200000| Untracked +| 25|0x0000000703200000, 0x0000000703400000, 0x0000000703400000|100%| O| |TAMS 0x0000000703400000, 0x0000000703400000| Untracked +| 26|0x0000000703400000, 0x0000000703565000, 0x0000000703600000| 69%| O| |TAMS 0x0000000703565000, 0x0000000703565000| Untracked +| 27|0x0000000703600000, 0x0000000703726e00, 0x0000000703800000| 57%| O| |TAMS 0x0000000703726e00, 0x0000000703726e00| Untracked +| 28|0x0000000703800000, 0x0000000703a00000, 0x0000000703a00000|100%| O| |TAMS 0x0000000703a00000, 0x0000000703a00000| Untracked +| 29|0x0000000703a00000, 0x0000000703c00000, 0x0000000703c00000|100%| O| |TAMS 0x0000000703c00000, 0x0000000703c00000| Untracked +| 30|0x0000000703c00000, 0x0000000703e00000, 0x0000000703e00000|100%| O| |TAMS 0x0000000703e00000, 0x0000000703e00000| Untracked +| 31|0x0000000703e00000, 0x0000000704000000, 0x0000000704000000|100%| O| |TAMS 0x0000000704000000, 0x0000000704000000| Untracked +| 32|0x0000000704000000, 0x0000000704200000, 0x0000000704200000|100%| O| |TAMS 0x0000000704200000, 0x0000000704200000| Untracked +| 33|0x0000000704200000, 0x0000000704400000, 0x0000000704400000|100%| O| |TAMS 0x0000000704400000, 0x0000000704400000| Untracked +| 34|0x0000000704400000, 0x0000000704600000, 0x0000000704600000|100%| O| |TAMS 0x0000000704600000, 0x0000000704600000| Untracked +| 35|0x0000000704600000, 0x0000000704800000, 0x0000000704800000|100%| O| |TAMS 0x0000000704800000, 0x0000000704800000| Untracked +| 36|0x0000000704800000, 0x0000000704a00000, 0x0000000704a00000|100%| O| |TAMS 0x0000000704a00000, 0x0000000704a00000| Untracked +| 37|0x0000000704a00000, 0x0000000704c00000, 0x0000000704c00000|100%| O| |TAMS 0x0000000704c00000, 0x0000000704c00000| Untracked +| 38|0x0000000704c00000, 0x0000000704e00000, 0x0000000704e00000|100%| O| |TAMS 0x0000000704e00000, 0x0000000704e00000| Untracked +| 39|0x0000000704e00000, 0x0000000705000000, 0x0000000705000000|100%| O| |TAMS 0x0000000705000000, 0x0000000705000000| Untracked +| 40|0x0000000705000000, 0x0000000705200000, 0x0000000705200000|100%| O| |TAMS 0x0000000705200000, 0x0000000705200000| Untracked +| 41|0x0000000705200000, 0x0000000705400000, 0x0000000705400000|100%| O| |TAMS 0x0000000705400000, 0x0000000705400000| Untracked +| 42|0x0000000705400000, 0x00000007055d3c00, 0x0000000705600000| 91%| O| |TAMS 0x00000007055d3c00, 0x00000007055d3c00| Untracked +| 43|0x0000000705600000, 0x0000000705800000, 0x0000000705800000|100%|HS| |TAMS 0x0000000705600000, 0x0000000705800000| Complete +| 44|0x0000000705800000, 0x0000000705a00000, 0x0000000705a00000|100%|HS| |TAMS 0x0000000705800000, 0x0000000705a00000| Complete +| 45|0x0000000705a00000, 0x0000000705c00000, 0x0000000705c00000|100%|HS| |TAMS 0x0000000705a00000, 0x0000000705c00000| Complete +| 46|0x0000000705c00000, 0x0000000705e00000, 0x0000000705e00000|100%|HS| |TAMS 0x0000000705c00000, 0x0000000705e00000| Complete +| 47|0x0000000705e00000, 0x0000000706000000, 0x0000000706000000|100%|HS| |TAMS 0x0000000705e00000, 0x0000000706000000| Complete +| 48|0x0000000706000000, 0x0000000706200000, 0x0000000706200000|100%|HS| |TAMS 0x0000000706000000, 0x0000000706200000| Complete +| 49|0x0000000706200000, 0x0000000706400000, 0x0000000706400000|100%|HS| |TAMS 0x0000000706200000, 0x0000000706400000| Complete +| 50|0x0000000706400000, 0x0000000706600000, 0x0000000706600000|100%|HS| |TAMS 0x0000000706400000, 0x0000000706600000| Complete +| 51|0x0000000706600000, 0x0000000706800000, 0x0000000706800000|100%|HS| |TAMS 0x0000000706600000, 0x0000000706800000| Complete +| 52|0x0000000706800000, 0x0000000706a00000, 0x0000000706a00000|100%|HS| |TAMS 0x0000000706800000, 0x0000000706a00000| Complete +| 53|0x0000000706a00000, 0x0000000706c00000, 0x0000000706c00000|100%| O| |TAMS 0x0000000706a00000, 0x0000000706c00000| Untracked +| 54|0x0000000706c00000, 0x0000000706e00000, 0x0000000706e00000|100%| O| |TAMS 0x0000000706c00000, 0x0000000706e00000| Untracked +| 55|0x0000000706e00000, 0x0000000707000000, 0x0000000707000000|100%| O| |TAMS 0x0000000706e00000, 0x0000000707000000| Untracked +| 56|0x0000000707000000, 0x0000000707200000, 0x0000000707200000|100%| O| |TAMS 0x0000000707000000, 0x0000000707200000| Untracked +| 57|0x0000000707200000, 0x0000000707400000, 0x0000000707400000|100%| O| |TAMS 0x0000000707200000, 0x0000000707400000| Untracked +| 58|0x0000000707400000, 0x0000000707600000, 0x0000000707600000|100%| O| |TAMS 0x0000000707400000, 0x0000000707600000| Untracked +| 59|0x0000000707600000, 0x0000000707800000, 0x0000000707800000|100%| O| |TAMS 0x0000000707600000, 0x0000000707800000| Untracked +| 60|0x0000000707800000, 0x000000070797ba00, 0x0000000707a00000| 74%| O| |TAMS 0x0000000707800000, 0x000000070797ba00| Untracked +| 61|0x0000000707a00000, 0x0000000707a00000, 0x0000000707c00000| 0%| F| |TAMS 0x0000000707a00000, 0x0000000707a00000| Untracked +| 62|0x0000000707c00000, 0x0000000707c00000, 0x0000000707e00000| 0%| F| |TAMS 0x0000000707c00000, 0x0000000707c00000| Untracked +| 63|0x0000000707e00000, 0x0000000708000000, 0x0000000708000000|100%| O| |TAMS 0x0000000707e00000, 0x0000000708000000| Untracked +| 64|0x0000000708000000, 0x0000000708200000, 0x0000000708200000|100%| O| |TAMS 0x0000000708000000, 0x0000000708200000| Untracked +| 65|0x0000000708200000, 0x0000000708400000, 0x0000000708400000|100%| O| |TAMS 0x0000000708200000, 0x0000000708400000| Untracked +| 66|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%|HS| |TAMS 0x0000000708400000, 0x0000000708400000| Complete +| 67|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%|HC| |TAMS 0x0000000708600000, 0x0000000708600000| Complete +| 68|0x0000000708800000, 0x0000000708a00000, 0x0000000708a00000|100%|HC| |TAMS 0x0000000708800000, 0x0000000708800000| Complete +| 69|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked +| 70|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked +| 71|0x0000000708e00000, 0x0000000709000000, 0x0000000709000000|100%| O| |TAMS 0x0000000708e00000, 0x0000000709000000| Untracked +| 72|0x0000000709000000, 0x0000000709200000, 0x0000000709200000|100%| O| |TAMS 0x0000000709000000, 0x0000000709200000| Untracked +| 73|0x0000000709200000, 0x0000000709400000, 0x0000000709400000|100%| O| |TAMS 0x0000000709200000, 0x0000000709400000| Untracked +| 74|0x0000000709400000, 0x0000000709600000, 0x0000000709600000|100%| O| |TAMS 0x0000000709400000, 0x0000000709600000| Untracked +| 75|0x0000000709600000, 0x0000000709800000, 0x0000000709800000|100%| O| |TAMS 0x0000000709600000, 0x0000000709800000| Untracked +| 76|0x0000000709800000, 0x0000000709a00000, 0x0000000709a00000|100%| O| |TAMS 0x0000000709800000, 0x0000000709a00000| Untracked +| 77|0x0000000709a00000, 0x0000000709c00000, 0x0000000709c00000|100%| O| |TAMS 0x0000000709a00000, 0x0000000709c00000| Untracked +| 78|0x0000000709c00000, 0x0000000709e00000, 0x0000000709e00000|100%| O| |TAMS 0x0000000709c00000, 0x0000000709e00000| Untracked +| 79|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked +| 80|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked +| 81|0x000000070a200000, 0x000000070a200000, 0x000000070a400000| 0%| F| |TAMS 0x000000070a200000, 0x000000070a200000| Untracked +| 82|0x000000070a400000, 0x000000070a400000, 0x000000070a600000| 0%| F| |TAMS 0x000000070a400000, 0x000000070a400000| Untracked +| 83|0x000000070a600000, 0x000000070a600000, 0x000000070a800000| 0%| F| |TAMS 0x000000070a600000, 0x000000070a600000| Untracked +| 84|0x000000070a800000, 0x000000070a800000, 0x000000070aa00000| 0%| F| |TAMS 0x000000070a800000, 0x000000070a800000| Untracked +| 85|0x000000070aa00000, 0x000000070aa00000, 0x000000070ac00000| 0%| F| |TAMS 0x000000070aa00000, 0x000000070aa00000| Untracked +| 86|0x000000070ac00000, 0x000000070ac00000, 0x000000070ae00000| 0%| F| |TAMS 0x000000070ac00000, 0x000000070ac00000| Untracked +| 87|0x000000070ae00000, 0x000000070ae00000, 0x000000070b000000| 0%| F| |TAMS 0x000000070ae00000, 0x000000070ae00000| Untracked +| 88|0x000000070b000000, 0x000000070b000000, 0x000000070b200000| 0%| F| |TAMS 0x000000070b000000, 0x000000070b000000| Untracked +| 89|0x000000070b200000, 0x000000070b200000, 0x000000070b400000| 0%| F| |TAMS 0x000000070b200000, 0x000000070b200000| Untracked +| 90|0x000000070b400000, 0x000000070b400000, 0x000000070b600000| 0%| F| |TAMS 0x000000070b400000, 0x000000070b400000| Untracked +| 91|0x000000070b600000, 0x000000070b600000, 0x000000070b800000| 0%| F| |TAMS 0x000000070b600000, 0x000000070b600000| Untracked +| 92|0x000000070b800000, 0x000000070b800000, 0x000000070ba00000| 0%| F| |TAMS 0x000000070b800000, 0x000000070b800000| Untracked +| 93|0x000000070ba00000, 0x000000070ba00000, 0x000000070bc00000| 0%| F| |TAMS 0x000000070ba00000, 0x000000070ba00000| Untracked +| 94|0x000000070bc00000, 0x000000070bc00000, 0x000000070be00000| 0%| F| |TAMS 0x000000070bc00000, 0x000000070bc00000| Untracked +| 95|0x000000070be00000, 0x000000070be00000, 0x000000070c000000| 0%| F| |TAMS 0x000000070be00000, 0x000000070be00000| Untracked +| 96|0x000000070c000000, 0x000000070c000000, 0x000000070c200000| 0%| F| |TAMS 0x000000070c000000, 0x000000070c000000| Untracked +| 97|0x000000070c200000, 0x000000070c200000, 0x000000070c400000| 0%| F| |TAMS 0x000000070c200000, 0x000000070c200000| Untracked +| 98|0x000000070c400000, 0x000000070c400000, 0x000000070c600000| 0%| F| |TAMS 0x000000070c400000, 0x000000070c400000| Untracked +| 99|0x000000070c600000, 0x000000070c600000, 0x000000070c800000| 0%| F| |TAMS 0x000000070c600000, 0x000000070c600000| Untracked +| 100|0x000000070c800000, 0x000000070c800000, 0x000000070ca00000| 0%| F| |TAMS 0x000000070c800000, 0x000000070c800000| Untracked +| 101|0x000000070ca00000, 0x000000070ca00000, 0x000000070cc00000| 0%| F| |TAMS 0x000000070ca00000, 0x000000070ca00000| Untracked +| 102|0x000000070cc00000, 0x000000070cc00000, 0x000000070ce00000| 0%| F| |TAMS 0x000000070cc00000, 0x000000070cc00000| Untracked +| 103|0x000000070ce00000, 0x000000070ce00000, 0x000000070d000000| 0%| F| |TAMS 0x000000070ce00000, 0x000000070ce00000| Untracked +| 104|0x000000070d000000, 0x000000070d000000, 0x000000070d200000| 0%| F| |TAMS 0x000000070d000000, 0x000000070d000000| Untracked +| 105|0x000000070d200000, 0x000000070d200000, 0x000000070d400000| 0%| F| |TAMS 0x000000070d200000, 0x000000070d200000| Untracked +| 106|0x000000070d400000, 0x000000070d400000, 0x000000070d600000| 0%| F| |TAMS 0x000000070d400000, 0x000000070d400000| Untracked +| 107|0x000000070d600000, 0x000000070d600000, 0x000000070d800000| 0%| F| |TAMS 0x000000070d600000, 0x000000070d600000| Untracked +| 108|0x000000070d800000, 0x000000070d800000, 0x000000070da00000| 0%| F| |TAMS 0x000000070d800000, 0x000000070d800000| Untracked +| 109|0x000000070da00000, 0x000000070da00000, 0x000000070dc00000| 0%| F| |TAMS 0x000000070da00000, 0x000000070da00000| Untracked +| 110|0x000000070dc00000, 0x000000070dc00000, 0x000000070de00000| 0%| F| |TAMS 0x000000070dc00000, 0x000000070dc00000| Untracked +| 111|0x000000070de00000, 0x000000070de00000, 0x000000070e000000| 0%| F| |TAMS 0x000000070de00000, 0x000000070de00000| Untracked +| 112|0x000000070e000000, 0x000000070e000000, 0x000000070e200000| 0%| F| |TAMS 0x000000070e000000, 0x000000070e000000| Untracked +| 113|0x000000070e200000, 0x000000070e200000, 0x000000070e400000| 0%| F| |TAMS 0x000000070e200000, 0x000000070e200000| Untracked +| 114|0x000000070e400000, 0x000000070e400000, 0x000000070e600000| 0%| F| |TAMS 0x000000070e400000, 0x000000070e400000| Untracked +| 115|0x000000070e600000, 0x000000070e600000, 0x000000070e800000| 0%| F| |TAMS 0x000000070e600000, 0x000000070e600000| Untracked +| 116|0x000000070e800000, 0x000000070e800000, 0x000000070ea00000| 0%| F| |TAMS 0x000000070e800000, 0x000000070e800000| Untracked +| 117|0x000000070ea00000, 0x000000070ea00000, 0x000000070ec00000| 0%| F| |TAMS 0x000000070ea00000, 0x000000070ea00000| Untracked +| 118|0x000000070ec00000, 0x000000070ec00000, 0x000000070ee00000| 0%| F| |TAMS 0x000000070ec00000, 0x000000070ec00000| Untracked +| 119|0x000000070ee00000, 0x000000070ee00000, 0x000000070f000000| 0%| F| |TAMS 0x000000070ee00000, 0x000000070ee00000| Untracked +| 120|0x000000070f000000, 0x000000070f000000, 0x000000070f200000| 0%| F| |TAMS 0x000000070f000000, 0x000000070f000000| Untracked +| 121|0x000000070f200000, 0x000000070f200000, 0x000000070f400000| 0%| F| |TAMS 0x000000070f200000, 0x000000070f200000| Untracked +| 122|0x000000070f400000, 0x000000070f400000, 0x000000070f600000| 0%| F| |TAMS 0x000000070f400000, 0x000000070f400000| Untracked +| 123|0x000000070f600000, 0x000000070f600000, 0x000000070f800000| 0%| F| |TAMS 0x000000070f600000, 0x000000070f600000| Untracked +| 124|0x000000070f800000, 0x000000070f800000, 0x000000070fa00000| 0%| F| |TAMS 0x000000070f800000, 0x000000070f800000| Untracked +| 125|0x000000070fa00000, 0x000000070fa00000, 0x000000070fc00000| 0%| F| |TAMS 0x000000070fa00000, 0x000000070fa00000| Untracked +| 126|0x000000070fc00000, 0x000000070fc00000, 0x000000070fe00000| 0%| F| |TAMS 0x000000070fc00000, 0x000000070fc00000| Untracked +| 127|0x000000070fe00000, 0x000000070fe00000, 0x0000000710000000| 0%| F| |TAMS 0x000000070fe00000, 0x000000070fe00000| Untracked +| 128|0x0000000710000000, 0x0000000710000000, 0x0000000710200000| 0%| F| |TAMS 0x0000000710000000, 0x0000000710000000| Untracked +| 129|0x0000000710200000, 0x0000000710200000, 0x0000000710400000| 0%| F| |TAMS 0x0000000710200000, 0x0000000710200000| Untracked +| 130|0x0000000710400000, 0x0000000710400000, 0x0000000710600000| 0%| F| |TAMS 0x0000000710400000, 0x0000000710400000| Untracked +| 131|0x0000000710600000, 0x0000000710600000, 0x0000000710800000| 0%| F| |TAMS 0x0000000710600000, 0x0000000710600000| Untracked +| 132|0x0000000710800000, 0x0000000710800000, 0x0000000710a00000| 0%| F| |TAMS 0x0000000710800000, 0x0000000710800000| Untracked +| 133|0x0000000710a00000, 0x0000000710a00000, 0x0000000710c00000| 0%| F| |TAMS 0x0000000710a00000, 0x0000000710a00000| Untracked +| 134|0x0000000710c00000, 0x0000000710c00000, 0x0000000710e00000| 0%| F| |TAMS 0x0000000710c00000, 0x0000000710c00000| Untracked +| 135|0x0000000710e00000, 0x0000000710e00000, 0x0000000711000000| 0%| F| |TAMS 0x0000000710e00000, 0x0000000710e00000| Untracked +| 136|0x0000000711000000, 0x0000000711000000, 0x0000000711200000| 0%| F| |TAMS 0x0000000711000000, 0x0000000711000000| Untracked +| 137|0x0000000711200000, 0x0000000711200000, 0x0000000711400000| 0%| F| |TAMS 0x0000000711200000, 0x0000000711200000| Untracked +| 138|0x0000000711400000, 0x0000000711400000, 0x0000000711600000| 0%| F| |TAMS 0x0000000711400000, 0x0000000711400000| Untracked +| 139|0x0000000711600000, 0x0000000711600000, 0x0000000711800000| 0%| F| |TAMS 0x0000000711600000, 0x0000000711600000| Untracked +| 140|0x0000000711800000, 0x0000000711800000, 0x0000000711a00000| 0%| F| |TAMS 0x0000000711800000, 0x0000000711800000| Untracked +| 141|0x0000000711a00000, 0x0000000711a00000, 0x0000000711c00000| 0%| F| |TAMS 0x0000000711a00000, 0x0000000711a00000| Untracked +| 142|0x0000000711c00000, 0x0000000711c00000, 0x0000000711e00000| 0%| F| |TAMS 0x0000000711c00000, 0x0000000711c00000| Untracked +| 143|0x0000000711e00000, 0x0000000711e00000, 0x0000000712000000| 0%| F| |TAMS 0x0000000711e00000, 0x0000000711e00000| Untracked +| 144|0x0000000712000000, 0x0000000712000000, 0x0000000712200000| 0%| F| |TAMS 0x0000000712000000, 0x0000000712000000| Untracked +| 145|0x0000000712200000, 0x0000000712200000, 0x0000000712400000| 0%| F| |TAMS 0x0000000712200000, 0x0000000712200000| Untracked +| 146|0x0000000712400000, 0x0000000712400000, 0x0000000712600000| 0%| F| |TAMS 0x0000000712400000, 0x0000000712400000| Untracked +| 147|0x0000000712600000, 0x0000000712600000, 0x0000000712800000| 0%| F| |TAMS 0x0000000712600000, 0x0000000712600000| Untracked +| 148|0x0000000712800000, 0x0000000712800000, 0x0000000712a00000| 0%| F| |TAMS 0x0000000712800000, 0x0000000712800000| Untracked +| 149|0x0000000712a00000, 0x0000000712a00000, 0x0000000712c00000| 0%| F| |TAMS 0x0000000712a00000, 0x0000000712a00000| Untracked +| 150|0x0000000712c00000, 0x0000000712c00000, 0x0000000712e00000| 0%| F| |TAMS 0x0000000712c00000, 0x0000000712c00000| Untracked +| 151|0x0000000712e00000, 0x0000000712e00000, 0x0000000713000000| 0%| F| |TAMS 0x0000000712e00000, 0x0000000712e00000| Untracked +| 152|0x0000000713000000, 0x0000000713000000, 0x0000000713200000| 0%| F| |TAMS 0x0000000713000000, 0x0000000713000000| Untracked +| 153|0x0000000713200000, 0x00000007133a4680, 0x0000000713400000| 82%| S|CS|TAMS 0x0000000713200000, 0x0000000713200000| Complete +| 154|0x0000000713400000, 0x0000000713600000, 0x0000000713600000|100%| S|CS|TAMS 0x0000000713400000, 0x0000000713400000| Complete +| 155|0x0000000713600000, 0x0000000713800000, 0x0000000713800000|100%| S|CS|TAMS 0x0000000713600000, 0x0000000713600000| Complete +| 156|0x0000000713800000, 0x0000000713a00000, 0x0000000713a00000|100%| S|CS|TAMS 0x0000000713800000, 0x0000000713800000| Complete +| 157|0x0000000713a00000, 0x0000000713b14ee8, 0x0000000713c00000| 54%| E| |TAMS 0x0000000713a00000, 0x0000000713a00000| Complete +| 158|0x0000000713c00000, 0x0000000713e00000, 0x0000000713e00000|100%| E|CS|TAMS 0x0000000713c00000, 0x0000000713c00000| Complete +| 159|0x0000000713e00000, 0x0000000714000000, 0x0000000714000000|100%| E|CS|TAMS 0x0000000713e00000, 0x0000000713e00000| Complete +| 160|0x0000000714000000, 0x0000000714200000, 0x0000000714200000|100%| E|CS|TAMS 0x0000000714000000, 0x0000000714000000| Complete +| 161|0x0000000714200000, 0x0000000714400000, 0x0000000714400000|100%| E|CS|TAMS 0x0000000714200000, 0x0000000714200000| Complete +| 334|0x0000000729c00000, 0x0000000729e00000, 0x0000000729e00000|100%| E|CS|TAMS 0x0000000729c00000, 0x0000000729c00000| Complete +| 335|0x0000000729e00000, 0x000000072a000000, 0x000000072a000000|100%| E|CS|TAMS 0x0000000729e00000, 0x0000000729e00000| Complete +| 440|0x0000000737000000, 0x0000000737200000, 0x0000000737200000|100%| E|CS|TAMS 0x0000000737000000, 0x0000000737000000| Complete +| 441|0x0000000737200000, 0x0000000737400000, 0x0000000737400000|100%| E|CS|TAMS 0x0000000737200000, 0x0000000737200000| Complete +| 442|0x0000000737400000, 0x0000000737600000, 0x0000000737600000|100%| E|CS|TAMS 0x0000000737400000, 0x0000000737400000| Complete +| 767|0x000000075fe00000, 0x0000000760000000, 0x0000000760000000|100%| E|CS|TAMS 0x000000075fe00000, 0x000000075fe00000| Complete +|2046|0x00000007ffc00000, 0x00000007ffd74000, 0x00000007ffe00000| 72%|OA| |TAMS 0x00000007ffd74000, 0x00000007ffd74000| Untracked +|2047|0x00000007ffe00000, 0x00000007ffe74000, 0x0000000800000000| 22%|CA| |TAMS 0x00000007ffe74000, 0x00000007ffe74000| Untracked + +Card table byte_map: [0x0000000122604000,0x0000000122e04000] _byte_map_base: 0x000000011ee04000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x0000000104f136f0, (CMBitMap*) 0x0000000104f13730 + Prev Bits: [0x0000000123604000, 0x0000000127604000) + Next Bits: [0x0000000127604000, 0x000000012b604000) + +Polling page: 0x0000000104824000 + +Metaspace: + +Usage: + Non-class: 130.16 MB used. + Class: 17.80 MB used. + Both: 147.95 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 130.81 MB ( 68%) committed, 3 nodes. + Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 149.19 MB ( 12%) committed. + +Chunk freelists: + Non-Class: 12.97 MB + Class: 13.67 MB + Both: 26.64 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 164.31 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 14. +num_arena_births: 2088. +num_arena_deaths: 0. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2384. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 14. +num_chunks_taken_from_freelist: 7782. +num_chunk_merges: 11. +num_chunk_splits: 5822. +num_chunks_enlarged: 4401. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=8986Kb max_used=8986Kb free=110182Kb + bounds [0x000000011a9e8000, 0x000000011b2c8000, 0x0000000121e48000] +CodeHeap 'profiled nmethods': size=119152Kb used=19697Kb max_used=20086Kb free=99454Kb + bounds [0x0000000112e48000, 0x00000001141e8000, 0x000000011a2a4000] +CodeHeap 'non-nmethods': size=7440Kb used=3323Kb max_used=3467Kb free=4116Kb + bounds [0x000000011a2a4000, 0x000000011a624000, 0x000000011a9e8000] + total_blobs=11917 nmethods=10855 adapters=973 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 20.973 Thread 0x0000000b668e6400 14759 3 org.apache.comet.serde.Config$ConfigMap$Builder::putEntries (87 bytes) +Event: 20.973 Thread 0x0000000b66930000 nmethod 14754 0x00000001138b2b10 code [0x00000001138b2cc0, 0x00000001138b2eb8] +Event: 20.973 Thread 0x0000000b66930000 14763 3 com.google.protobuf.MapEntry::writeTo (26 bytes) +Event: 20.973 Thread 0x0000000b66930000 nmethod 14763 0x00000001138b2610 code [0x00000001138b27c0, 0x00000001138b29b8] +Event: 20.973 Thread 0x0000000b66930000 14764 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) +Event: 20.973 Thread 0x0000000b668e6400 nmethod 14759 0x00000001138b1b90 code [0x00000001138b1dc0, 0x00000001138b2348] +Event: 20.973 Thread 0x0000000b668e6400 14766 3 org.apache.comet.ConfigEntry::key (13 bytes) +Event: 20.973 Thread 0x0000000b668e6400 nmethod 14766 0x00000001138b1710 code [0x00000001138b18c0, 0x00000001138b1a48] +Event: 20.973 Thread 0x0000000b66930000 nmethod 14764 0x00000001138b1110 code [0x00000001138b1300, 0x00000001138b1568] +Event: 20.973 Thread 0x0000000b66930000 14762 3 org.apache.logging.log4j.message.AbstractMessageFactory::newMessage (17 bytes) +Event: 20.973 Thread 0x0000000b668e6400 14765 3 org.apache.logging.log4j.message.SimpleMessage:: (23 bytes) +Event: 20.973 Thread 0x0000000b668e6400 nmethod 14765 0x00000001138b0b90 code [0x00000001138b0d40, 0x00000001138b0fc8] +Event: 20.973 Thread 0x0000000b668e7600 nmethod 14758 0x00000001138afc10 code [0x00000001138afec0, 0x00000001138b0778] +Event: 20.973 Thread 0x0000000b66930000 nmethod 14762 0x00000001138af690 code [0x00000001138af840, 0x00000001138afa78] +Event: 20.973 Thread 0x0000000b65684000 nmethod 14756 0x0000000113dd5f10 code [0x0000000113dd65c0, 0x0000000113dd99e8] +Event: 20.973 Thread 0x0000000b668e7600 14767 3 scala.collection.immutable.Map$EmptyMap$::updated (21 bytes) +Event: 20.973 Thread 0x0000000b668e7600 nmethod 14767 0x00000001138af110 code [0x00000001138af2c0, 0x00000001138af4f8] +Event: 20.974 Thread 0x0000000b65684000 14768 3 java.util.stream.ReferencePipeline$2$1::begin (13 bytes) +Event: 20.974 Thread 0x0000000b668e6400 14769 3 java.util.Optional::isPresent (13 bytes) +Event: 20.974 Thread 0x0000000b65684000 nmethod 14768 0x00000001138aec90 code [0x00000001138aee40, 0x00000001138af048] + +GC Heap History (20 events): +Event: 7.381 GC heap before +{Heap before GC invocations=27 (full 0): + garbage-first heap total 348160K, used 283997K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 66 young (135168K), 2 survivors (4096K) + Metaspace used 140795K, committed 141760K, reserved 1179648K + class space used 16877K, committed 17344K, reserved 1048576K +} +Event: 7.382 GC heap after +{Heap after GC invocations=28 (full 0): + garbage-first heap total 348160K, used 152280K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 140795K, committed 141760K, reserved 1179648K + class space used 16877K, committed 17344K, reserved 1048576K +} +Event: 7.558 GC heap before +{Heap before GC invocations=28 (full 0): + garbage-first heap total 348160K, used 289496K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 68 young (139264K), 3 survivors (6144K) + Metaspace used 147451K, committed 148544K, reserved 1179648K + class space used 17566K, committed 18048K, reserved 1048576K +} +Event: 7.559 GC heap after +{Heap after GC invocations=29 (full 0): + garbage-first heap total 348160K, used 154116K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 147451K, committed 148544K, reserved 1179648K + class space used 17566K, committed 18048K, reserved 1048576K +} +Event: 7.889 GC heap before +{Heap before GC invocations=29 (full 0): + garbage-first heap total 348160K, used 289284K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 70 young (143360K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 7.891 GC heap after +{Heap after GC invocations=30 (full 0): + garbage-first heap total 348160K, used 154509K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.181 GC heap before +{Heap before GC invocations=30 (full 0): + garbage-first heap total 348160K, used 291725K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 71 young (145408K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.183 GC heap after +{Heap after GC invocations=31 (full 0): + garbage-first heap total 348160K, used 154662K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.466 GC heap before +{Heap before GC invocations=31 (full 0): + garbage-first heap total 348160K, used 295974K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 73 young (149504K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.467 GC heap after +{Heap after GC invocations=32 (full 0): + garbage-first heap total 348160K, used 154558K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.753 GC heap before +{Heap before GC invocations=32 (full 0): + garbage-first heap total 348160K, used 297918K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 74 young (151552K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 8.754 GC heap after +{Heap after GC invocations=33 (full 0): + garbage-first heap total 348160K, used 154400K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 9.049 GC heap before +{Heap before GC invocations=33 (full 0): + garbage-first heap total 348160K, used 299808K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 75 young (153600K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 9.050 GC heap after +{Heap after GC invocations=34 (full 0): + garbage-first heap total 348160K, used 154646K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148885K, committed 150016K, reserved 1245184K + class space used 17782K, committed 18304K, reserved 1048576K +} +Event: 20.862 GC heap before +{Heap before GC invocations=34 (full 0): + garbage-first heap total 348160K, used 302102K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 75 young (153600K), 4 survivors (8192K) + Metaspace used 149316K, committed 150400K, reserved 1245184K + class space used 17855K, committed 18368K, reserved 1048576K +} +Event: 20.864 GC heap after +{Heap after GC invocations=35 (full 0): + garbage-first heap total 348160K, used 154622K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 149316K, committed 150400K, reserved 1245184K + class space used 17855K, committed 18368K, reserved 1048576K +} +Event: 20.887 GC heap before +{Heap before GC invocations=35 (full 0): + garbage-first heap total 348160K, used 160766K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 6 young (12288K), 4 survivors (8192K) + Metaspace used 149347K, committed 150464K, reserved 1245184K + class space used 17855K, committed 18368K, reserved 1048576K +} +Event: 20.889 GC heap after +{Heap after GC invocations=36 (full 0): + garbage-first heap total 348160K, used 154628K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 149347K, committed 150464K, reserved 1245184K + class space used 17855K, committed 18368K, reserved 1048576K +} +Event: 20.938 GC heap before +{Heap before GC invocations=36 (full 0): + garbage-first heap total 348160K, used 168964K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 11 young (22528K), 4 survivors (8192K) + Metaspace used 150217K, committed 151360K, reserved 1245184K + class space used 17979K, committed 18496K, reserved 1048576K +} +Event: 20.940 GC heap after +{Heap after GC invocations=37 (full 0): + garbage-first heap total 348160K, used 155038K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 150217K, committed 151360K, reserved 1245184K + class space used 17979K, committed 18496K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.443 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li +Event: 3.444 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.444 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.423 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib +Event: 5.936 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 5.936 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 5.936 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 5.936 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 5.936 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 5.936 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 5.936 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.936 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 5.936 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 5.936 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.936 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 5.936 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 5.937 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 5.937 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 6.131 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-11281734455283120293.dylib +Event: 6.834 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-2574e9d4-69d6-4911-a16f-7c6239f501cb-libsnappyjava.dylib + +Deoptimization events (20 events): +Event: 20.964 Thread 0x0000000b661e1800 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b21a33c relative=0x000000000000013c +Event: 20.964 Thread 0x0000000b661e2400 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 20.964 Thread 0x0000000b661e1e00 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017cdc8ef0 mode 2 +Event: 20.964 Thread 0x0000000b661e1800 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 20.964 Thread 0x0000000b661e2400 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017d1d4ff0 +Event: 20.964 Thread 0x0000000b661e1800 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017c9bcff0 +Event: 20.964 Thread 0x0000000b661e2a00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b21a33c relative=0x000000000000013c +Event: 20.964 Thread 0x0000000b661e2a00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 20.964 Thread 0x0000000b661e2400 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017d1d4ef0 mode 2 +Event: 20.964 Thread 0x0000000b661e2a00 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017d5e0ff0 +Event: 20.964 Thread 0x0000000b661e1800 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c9bcef0 mode 2 +Event: 20.964 Thread 0x0000000b661e2a00 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017d5e0ef0 mode 2 +Event: 20.964 Thread 0x0000000b661e1200 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011af9baa8 relative=0x00000000000000e8 +Event: 20.964 Thread 0x0000000b661e1200 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011af9baa8 method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 20.964 Thread 0x0000000b661e1200 DEOPT PACKING pc=0x000000011af9baa8 sp=0x000000017c5b0f70 +Event: 20.964 Thread 0x0000000b661e1200 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c5b0ef0 mode 2 +Event: 20.968 Thread 0x0000000b661e1800 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011acf1e10 relative=0x0000000000000810 +Event: 20.968 Thread 0x0000000b661e1800 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011acf1e10 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 +Event: 20.968 Thread 0x0000000b661e1800 DEOPT PACKING pc=0x000000011acf1e10 sp=0x000000017c9bc850 +Event: 20.968 Thread 0x0000000b661e1800 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c9bc6a0 mode 2 + +Classes loaded (20 events): +Event: 7.527 Loading class java/lang/Class$EnclosingMethodInfo +Event: 7.527 Loading class java/lang/Class$EnclosingMethodInfo done +Event: 18.268 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper +Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$Unmapper +Event: 18.269 Loading class jdk/internal/access/foreign/UnmapperProxy +Event: 18.269 Loading class jdk/internal/access/foreign/UnmapperProxy done +Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$Unmapper done +Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done +Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask +Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask +Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done +Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done +Event: 18.271 Loading class java/util/function/LongBinaryOperator +Event: 18.271 Loading class java/util/function/LongBinaryOperator done +Event: 20.844 Loading class sun/nio/ch/ChannelInputStream +Event: 20.844 Loading class sun/nio/ch/ChannelInputStream done +Event: 20.903 Loading class java/math/BigDecimal$StringBuilderHelper +Event: 20.903 Loading class java/math/BigDecimal$StringBuilderHelper done +Event: 20.969 Loading class java/nio/channels/Channels$ReadableByteChannelImpl +Event: 20.969 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 20.964 Thread 0x0000000b661e1800 Exception (0x0000000714144c00) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.965 Thread 0x0000000b661e2a00 Exception (0x0000000714389228) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.965 Thread 0x0000000b661e2400 Exception (0x0000000729e6e3d8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.965 Thread 0x0000000b661e1200 Exception (0x00000007140f2820) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.965 Thread 0x0000000b661e1800 Exception (0x0000000714148218) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ebdf18) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000714106a28) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000714195898) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x000000071439d3f8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e82960) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000713f0c988) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ec1ef0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000714199670) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x00000007143a11d0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e86738) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ec57d0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e89fc8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000713f10218) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000713f93530) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x0000000713c00830) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] + +VM Operations (20 events): +Event: 8.466 Executing VM operation: G1CollectForAllocation +Event: 8.467 Executing VM operation: G1CollectForAllocation done +Event: 8.753 Executing VM operation: G1CollectForAllocation +Event: 8.754 Executing VM operation: G1CollectForAllocation done +Event: 9.049 Executing VM operation: G1CollectForAllocation +Event: 9.050 Executing VM operation: G1CollectForAllocation done +Event: 12.065 Executing VM operation: Cleanup +Event: 12.065 Executing VM operation: Cleanup done +Event: 19.099 Executing VM operation: Cleanup +Event: 19.099 Executing VM operation: Cleanup done +Event: 20.104 Executing VM operation: Cleanup +Event: 20.104 Executing VM operation: Cleanup done +Event: 20.862 Executing VM operation: G1TryInitiateConcMark +Event: 20.864 Executing VM operation: G1TryInitiateConcMark done +Event: 20.887 Executing VM operation: G1TryInitiateConcMark +Event: 20.889 Executing VM operation: G1TryInitiateConcMark done +Event: 20.929 Executing VM operation: ICBufferFull +Event: 20.929 Executing VM operation: ICBufferFull done +Event: 20.938 Executing VM operation: G1TryInitiateConcMark +Event: 20.940 Executing VM operation: G1TryInitiateConcMark done + +Memory protections (20 events): +Event: 7.391 Protecting memory [0x000000017f240000,0x000000017f24c000] with protection modes 0 +Event: 7.391 Protecting memory [0x000000017f64c000,0x000000017f658000] with protection modes 0 +Event: 9.328 Protecting memory [0x000000016fac8000,0x000000016fad4000] with protection modes 3 +Event: 9.328 Protecting memory [0x000000016f8bc000,0x000000016f8c8000] with protection modes 3 +Event: 9.328 Protecting memory [0x000000016ea74000,0x000000016ea80000] with protection modes 3 +Event: 9.331 Protecting memory [0x000000016f098000,0x000000016f0a4000] with protection modes 3 +Event: 14.336 Protecting memory [0x000000016ee8c000,0x000000016ee98000] with protection modes 3 +Event: 14.336 Protecting memory [0x000000016e868000,0x000000016e874000] with protection modes 3 +Event: 14.336 Protecting memory [0x000000016ec80000,0x000000016ec8c000] with protection modes 3 +Event: 20.846 Protecting memory [0x000000016e868000,0x000000016e874000] with protection modes 0 +Event: 20.856 Protecting memory [0x000000016ea74000,0x000000016ea80000] with protection modes 0 +Event: 20.856 Protecting memory [0x000000016ec80000,0x000000016ec8c000] with protection modes 0 +Event: 20.856 Protecting memory [0x000000016ee8c000,0x000000016ee98000] with protection modes 0 +Event: 20.874 Protecting memory [0x000000016f098000,0x000000016f0a4000] with protection modes 0 +Event: 20.893 Protecting memory [0x000000016f8bc000,0x000000016f8c8000] with protection modes 0 +Event: 20.941 Protecting memory [0x000000030041c000,0x0000000300428000] with protection modes 0 +Event: 20.941 Protecting memory [0x0000000300828000,0x0000000300834000] with protection modes 0 +Event: 20.941 Protecting memory [0x0000000300e58000,0x0000000300e64000] with protection modes 0 +Event: 20.942 Protecting memory [0x0000000301264000,0x0000000301270000] with protection modes 0 +Event: 20.942 Protecting memory [0x0000000301670000,0x000000030167c000] with protection modes 0 + +Nmethod flushes (20 events): +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416de90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416ec10 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416f310 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416fb90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114170d90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114171c10 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114172d90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114173890 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114175990 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114176d10 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114177f90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114178410 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114178d90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114179810 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114179c90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011417af90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011417f610 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114180a90 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114181590 +Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114182710 + +Events (20 events): +Event: 7.391 Thread 0x0000000b64460c00 Thread added: 0x0000000b64461200 +Event: 7.391 Thread 0x0000000b64461200 Thread added: 0x0000000b64461800 +Event: 9.328 Thread 0x0000000b661e3600 Thread exited: 0x0000000b661e3600 +Event: 9.328 Thread 0x0000000b66930000 Thread exited: 0x0000000b66930000 +Event: 9.328 Thread 0x0000000b65684000 Thread exited: 0x0000000b65684000 +Event: 9.331 Thread 0x0000000b6619b000 Thread exited: 0x0000000b6619b000 +Event: 14.336 Thread 0x0000000b65ca3600 Thread exited: 0x0000000b65ca3600 +Event: 14.336 Thread 0x0000000b65ca0600 Thread exited: 0x0000000b65ca0600 +Event: 14.336 Thread 0x0000000b668e7600 Thread exited: 0x0000000b668e7600 +Event: 20.846 Thread 0x0000000b668e6400 Thread added: 0x0000000b6451f600 +Event: 20.856 Thread 0x0000000b668e6400 Thread added: 0x0000000b668e7600 +Event: 20.856 Thread 0x0000000b668e6400 Thread added: 0x0000000b65684000 +Event: 20.856 Thread 0x0000000b668e7600 Thread added: 0x0000000b66930000 +Event: 20.874 Thread 0x0000000b6451f600 Thread added: 0x0000000b66b0e400 +Event: 20.893 Thread 0x0000000b65095800 Thread added: 0x0000000b66198000 +Event: 20.941 Thread 0x0000000b65529200 Thread added: 0x0000000b661e3600 +Event: 20.941 Thread 0x0000000b661e3600 Thread added: 0x0000000b6451c600 +Event: 20.941 Thread 0x0000000b661e3600 Thread added: 0x0000000b64463600 +Event: 20.941 Thread 0x0000000b64463600 Thread added: 0x0000000b63d64000 +Event: 20.942 Thread 0x0000000b63d64000 Thread added: 0x0000000b63d64600 + + +Dynamic libraries: +0x000000010483c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x000000010a304000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x0000000104864000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x0000000104894000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x00000001048e4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x0000000104980000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x00000001049a8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x00000001049c8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x00000001049ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x000000010496c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x0000000104a08000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x0000000104a1c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x0000000109e7c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000149dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib +0x0000000109f4c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-11281734455283120293.dylib +0x0000000109f78000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-2574e9d4-69d6-4911-a16f-7c6239f501cb-libsnappyjava.dylib + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:39 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 8.72 7.58 5.42 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(863632k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/hs_err_pid59157.log b/spark/hs_err_pid59157.log new file mode 100644 index 0000000000..8660ac9ed5 --- /dev/null +++ b/spark/hs_err_pid59157.log @@ -0,0 +1,3785 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x0000000108ca9530, pid=59157, tid=29447 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 08:59:18 2026 MDT elapsed time: 44.767401 seconds (0d 0h 0m 44s) + +--------------- T H R E A D --------------- + +Current thread (0x0000000c67a9a400): JavaThread "Executor task launch worker for task 0.0 in stage 268.0 (TID 1092)" daemon [_thread_in_vm, id=29447, stack(0x000000017de60000,0x000000017e263000)] + +Stack: [0x000000017de60000,0x000000017e263000], sp=0x000000017e25a520, free space=4073k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 +C [libcomet-1214774261196833962.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 +C [libcomet-1214774261196833962.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc +C [libcomet-1214774261196833962.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 +C [libcomet-1214774261196833962.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 +C [libcomet-1214774261196833962.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c +C [libcomet-1214774261196833962.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 +C [libcomet-1214774261196833962.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 +C [libcomet-1214774261196833962.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 +C [libcomet-1214774261196833962.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 +C [libcomet-1214774261196833962.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 +C [libcomet-1214774261196833962.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 +C [libcomet-1214774261196833962.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 +C [libcomet-1214774261196833962.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c +C [libcomet-1214774261196833962.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 +C [libcomet-1214774261196833962.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 +C [libcomet-1214774261196833962.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c +C [libcomet-1214774261196833962.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 +C [libcomet-1214774261196833962.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 +C [libcomet-1214774261196833962.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 +C [libcomet-1214774261196833962.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c +C [libcomet-1214774261196833962.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 +C [libcomet-1214774261196833962.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 +C [libcomet-1214774261196833962.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 +C [libcomet-1214774261196833962.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c +C [libcomet-1214774261196833962.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c +C [libcomet-1214774261196833962.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-1214774261196833962.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-1214774261196833962.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-1214774261196833962.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-1214774261196833962.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-1214774261196833962.dylib+0x106628] __rust_try+0x20 +C [libcomet-1214774261196833962.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-1214774261196833962.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-1214774261196833962.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-1214774261196833962.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +J 17175 org.apache.comet.Native.executePlan(IIJ[J[J)J (0 bytes) @ 0x000000011984a554 [0x000000011984a4c0+0x0000000000000094] +J 18191 c1 org.apache.comet.vector.NativeUtil.allocateArrowStructs(I)Lscala/Tuple2; (69 bytes) @ 0x0000000112c2d368 [0x0000000112c2cb80+0x00000000000007e8] +J 18196 c1 org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option; (296 bytes) @ 0x0000000112c30e90 [0x0000000112c2fdc0+0x00000000000010d0] +J 24291 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option; (35 bytes) @ 0x00000001135ce88c [0x00000001135ce380+0x000000000000050c] +J 24290 c1 org.apache.comet.CometExecIterator$$Lambda$3979+0x000000c8021a53e8.apply()Ljava/lang/Object; (12 bytes) @ 0x00000001135cdf6c [0x00000001135cdec0+0x00000000000000ac] +J 24159 c1 org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object; (136 bytes) @ 0x0000000113589430 [0x00000001135891c0+0x0000000000000270] +J 24158 c1 org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option; (494 bytes) @ 0x000000011358b4f0 [0x000000011358aa80+0x0000000000000a70] +J 24297 c1 org.apache.comet.CometExecIterator.hasNext()Z (261 bytes) @ 0x00000001135d2384 [0x00000001135d1d80+0x0000000000000604] +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +J 33848 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus; (309 bytes) @ 0x0000000112892fa4 [0x0000000112890ac0+0x00000000000024e4] +J 33847 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object; (15 bytes) @ 0x0000000112fbca94 [0x0000000112fbc9c0+0x00000000000000d4] +J 30108 c1 org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object; (139 bytes) @ 0x0000000113ecd11c [0x0000000113eccfc0+0x000000000000015c] +J 26867 c1 org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object; (628 bytes) @ 0x0000000113819eb0 [0x00000001138183c0+0x0000000000001af0] +J 29617 c1 org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object; (102 bytes) @ 0x0000000113decc34 [0x0000000113dec940+0x00000000000002f4] +J 26396 c1 org.apache.spark.executor.Executor$TaskRunner$$Lambda$2518+0x000000c801df1868.apply()Ljava/lang/Object; (12 bytes) @ 0x000000011374af6c [0x000000011374aec0+0x00000000000000ac] +J 17578 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (402 bytes) @ 0x0000000112b3ec74 [0x0000000112b3eb00+0x0000000000000174] +J 21288 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000113036c14 [0x0000000113036b40+0x00000000000000d4] +J 17575 c1 org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000112b36b84 [0x0000000112b36ac0+0x00000000000000c4] +J 29509 c1 org.apache.spark.executor.Executor$TaskRunner.run()V (3920 bytes) @ 0x0000000113da7838 [0x0000000113da2e00+0x0000000000004a38] +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +J 17175 org.apache.comet.Native.executePlan(IIJ[J[J)J (0 bytes) @ 0x000000011984a554 [0x000000011984a4c0+0x0000000000000094] +J 24154 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J (34 bytes) @ 0x000000011358720c [0x00000001135870c0+0x000000000000014c] +J 24294 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object; (23 bytes) @ 0x00000001135cf684 [0x00000001135cf600+0x0000000000000084] +J 24296 c1 org.apache.comet.CometExecIterator$$Lambda$3982+0x000000c8021a67b8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object; (20 bytes) @ 0x00000001135cfdec [0x00000001135cfb40+0x00000000000002ac] +J 18196 c1 org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option; (296 bytes) @ 0x0000000112c30e90 [0x0000000112c2fdc0+0x00000000000010d0] +J 24291 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option; (35 bytes) @ 0x00000001135ce88c [0x00000001135ce380+0x000000000000050c] +J 24290 c1 org.apache.comet.CometExecIterator$$Lambda$3979+0x000000c8021a53e8.apply()Ljava/lang/Object; (12 bytes) @ 0x00000001135cdf6c [0x00000001135cdec0+0x00000000000000ac] +J 24159 c1 org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object; (136 bytes) @ 0x0000000113589430 [0x00000001135891c0+0x0000000000000270] +J 24158 c1 org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option; (494 bytes) @ 0x000000011358b4f0 [0x000000011358aa80+0x0000000000000a70] +J 24297 c1 org.apache.comet.CometExecIterator.hasNext()Z (261 bytes) @ 0x00000001135d2384 [0x00000001135d1d80+0x0000000000000604] +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +J 33848 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus; (309 bytes) @ 0x0000000112892fa4 [0x0000000112890ac0+0x00000000000024e4] +J 33847 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object; (15 bytes) @ 0x0000000112fbca94 [0x0000000112fbc9c0+0x00000000000000d4] +J 30108 c1 org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object; (139 bytes) @ 0x0000000113ecd11c [0x0000000113eccfc0+0x000000000000015c] +J 26867 c1 org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object; (628 bytes) @ 0x0000000113819eb0 [0x00000001138183c0+0x0000000000001af0] +J 29617 c1 org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object; (102 bytes) @ 0x0000000113decc34 [0x0000000113dec940+0x00000000000002f4] +J 26396 c1 org.apache.spark.executor.Executor$TaskRunner$$Lambda$2518+0x000000c801df1868.apply()Ljava/lang/Object; (12 bytes) @ 0x000000011374af6c [0x000000011374aec0+0x00000000000000ac] +J 17578 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (402 bytes) @ 0x0000000112b3ec74 [0x0000000112b3eb00+0x0000000000000174] +J 21288 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000113036c14 [0x0000000113036b40+0x00000000000000d4] +J 17575 c1 org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000112b36b84 [0x0000000112b36ac0+0x00000000000000c4] +J 29509 c1 org.apache.spark.executor.Executor$TaskRunner.run()V (3920 bytes) @ 0x0000000113da7838 [0x0000000113da2e00+0x0000000000004a38] +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 + +Registers: + x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000c67b00930 + x4=0x000000017e25a640 x5=0x0000000c67a9a400 x6=0x0000000c74e14300 x7=0xfffff0003ffff800 + x8=0x000000c801e60a98 x9=0x0000000000000000 x10=0x000000c800000000 x11=0x0000000000000004 +x12=0x000000013471cde0 x13=0x0000000000000000 x14=0xfffffffffffdb160 x15=0x0000000c69174000 +x16=0x0000000000000040 x17=0x0000000c690d2d10 x18=0x0000000000000000 x19=0x0000000c67a9a400 +x20=0x000000017e25a670 x21=0x000000017e25a640 x22=0x0000000c67054918 x23=0x0000000733506718 +x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000c67a9dac0 x27=0x0000000702bbdd88 +x28=0x0000000000000001 fp=0x000000017e25a630 lr=0x0000000108ca952c sp=0x000000017e25a520 +pc=0x0000000108ca9530 cpsr=0x0000000000001000 + +Register to memory mapping: + + x0=0x0 is NULL + x1=0x0000000000000007 is an unknown value + x2=0x0000000000000001 is an unknown value + x3=0x0000000c67b00930 points into unknown readable memory: 0x000000013471cde0 | e0 cd 71 34 01 00 00 00 + x4=0x000000017e25a640 is pointing into the stack for thread: 0x0000000c67a9a400 + x5=0x0000000c67a9a400 is a thread + x6=0x0000000c74e14300 points into unknown readable memory: 0x0000000c66005200 | 00 52 00 66 0c 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=0x000000c801e60a98 is pointing into metadata + x9=0x0 is NULL +x10=0x000000c800000000 is pointing into metadata +x11=0x0000000000000004 is an unknown value +x12={method} {0x000000013471cde0} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' +x13=0x0 is NULL +x14=0xfffffffffffdb160 is an unknown value +x15=0x0000000c69174000 points into unknown readable memory: 0xffffffffffffffff | ff ff ff ff ff ff ff ff +x16=0x0000000000000040 is an unknown value +x17=0x0000000c690d2d10 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff +x18=0x0 is NULL +x19=0x0000000c67a9a400 is a thread +x20=0x000000017e25a670 is pointing into the stack for thread: 0x0000000c67a9a400 +x21=0x000000017e25a640 is pointing into the stack for thread: 0x0000000c67a9a400 +x22=0x0000000c67054918 points into unknown readable memory: 0x0000000733506718 | 18 67 50 33 07 00 00 00 +x23=0x0000000733506718 is an oop: org.apache.comet.CometHandleBatchIterator +{0x0000000733506718} - klass: 'org/apache/comet/CometHandleBatchIterator' + - ---- fields (total size 2 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x00000007334fb620} (e669f6c4) +x24=0x0000000000000001 is an unknown value +x25=0x0 is NULL +x26=0x0000000c67a9dac0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x27=0x0000000702bbdd88 is an oop: [Z +{0x0000000702bbdd88} - klass: {type array bool} + - length: 9 +x28=0x0000000000000001 is an unknown value + + +Top of Stack: (sp=0x000000017e25a520) +0x000000017e25a520: 0000000c736e4000 0000000000000014 +0x000000017e25a530: 0000000c70134200 0000000000000014 +0x000000017e25a540: 0000000000000001 0000000000000000 +0x000000017e25a550: 000000017e25a587 0000000000000000 +0x000000017e25a560: 000000017e25a5a0 00000001587b40d0 +0x000000017e25a570: 0000000000000014 0000000c65f094d8 +0x000000017e25a580: 0000000000000014 0000000c65f094d8 +0x000000017e25a590: 0000000c65f094d8 0000000000000014 +0x000000017e25a5a0: 000000017e25a630 00000001587e3f58 +0x000000017e25a5b0: 0000000000000063 000000017e25a640 +0x000000017e25a5c0: 000000017e25a630 0000000108cc5f24 +0x000000017e25a5d0: 95e350fae7f500a8 0000000c67a9a400 +0x000000017e25a5e0: 0000000c67a9a400 0000000702bbdd88 +0x000000017e25a5f0: 0000000702b18548 0000000000000000 +0x000000017e25a600: 0000000c67a9a6b0 0000000000000008 +0x000000017e25a610: 0000000c6600a0a8 0000000c67b00930 +0x000000017e25a620: 0000000000000001 0000000c67a9a400 +0x000000017e25a630: 000000017e25a6e0 0000000108cad60c +0x000000017e25a640: 00000001093ee6b8 000000c80045a1d0 +0x000000017e25a650: 000000010000000a 0000000000000014 +0x000000017e25a660: 0000000000000000 0000000000000008 +0x000000017e25a670: 000000070000000a 000000073350ef48 +0x000000017e25a680: 0000000c67a9a400 0000000000000000 +0x000000017e25a690: 000000017e25a6e0 0000000151eb8c58 +0x000000017e25a6a0: 0000000702b18548 000000000000002f +0x000000017e25a6b0: 000000070483d960 0000000c638ed780 +0x000000017e25a6c0: 000000073350ef48 000000073350ef48 +0x000000017e25a6d0: 000000070483d960 000000000000002f +0x000000017e25a6e0: 000000017e25c2a0 0000000151e7d560 +0x000000017e25a6f0: 0000000000000000 000000017e25adc0 +0x000000017e25a700: 0000000c74935338 0000000000000038 +0x000000017e25a710: 0000000000000008 0000000159025fc8 + +Instructions: (pc=0x0000000108ca9530) +0x0000000108ca9430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 +0x0000000108ca9440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 +0x0000000108ca9450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 +0x0000000108ca9460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 +0x0000000108ca9470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 +0x0000000108ca9480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 +0x0000000108ca9490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 +0x0000000108ca94a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 +0x0000000108ca94b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa +0x0000000108ca94c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 +0x0000000108ca94d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 +0x0000000108ca94e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 +0x0000000108ca94f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a +0x0000000108ca9500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa +0x0000000108ca9510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 +0x0000000108ca9520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa +0x0000000108ca9530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 +0x0000000108ca9540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 +0x0000000108ca9550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 +0x0000000108ca9560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 +0x0000000108ca9570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 +0x0000000108ca9580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 +0x0000000108ca9590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa +0x0000000108ca95a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 +0x0000000108ca95b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 +0x0000000108ca95c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 +0x0000000108ca95d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 +0x0000000108ca95e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 +0x0000000108ca95f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa +0x0000000108ca9600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 +0x0000000108ca9610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 +0x0000000108ca9620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x0000000c736e4000 points into unknown readable memory: 0x0000000000000014 | 14 00 00 00 00 00 00 00 +stack at sp + 1 slots: 0x0000000000000014 is an unknown value +stack at sp + 2 slots: 0x0000000c70134200 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +stack at sp + 3 slots: 0x0000000000000014 is an unknown value +stack at sp + 4 slots: 0x0000000000000001 is an unknown value +stack at sp + 5 slots: 0x0 is NULL +stack at sp + 6 slots: 0x000000017e25a587 is pointing into the stack for thread: 0x0000000c67a9a400 +stack at sp + 7 slots: 0x0 is NULL + + +Compiled method (n/a) 44941 17175 n 0 org.apache.comet.Native::executePlan (native) + total in heap [0x000000011984a310,0x000000011984a688] = 888 + relocation [0x000000011984a468,0x000000011984a490] = 40 + main code [0x000000011984a4c0,0x000000011984a678] = 440 + oops [0x000000011984a678,0x000000011984a680] = 8 + metadata [0x000000011984a680,0x000000011984a688] = 8 + +[Constant Pool (empty)] + +[MachCode] +[Entry Point] + # {method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native' + # this: c_rarg1:c_rarg1 + = 'org/apache/comet/Native' + # parm0: c_rarg2 = int + # parm1: c_rarg3 = int + # parm2: c_rarg4:c_rarg4 + = long + # parm3: c_rarg5:c_rarg5 + = '[J' + # parm4: c_rarg6:c_rarg6 + = '[J' + # [sp+0x60] (sp of caller) + 0x000000011984a4c0: 2808 40b9 | 3f01 086b | 4000 0054 + + 0x000000011984a4cc: ; {runtime_call ic_miss_stub} + 0x000000011984a4cc: cd09 be17 +[Verified Entry Point] + 0x000000011984a4d0: 1f20 03d5 | e953 40d1 | 3f01 00f9 | fd7b bfa9 | fd03 0091 | ff43 01d1 | e617 00f9 | df00 1feb + 0x000000011984a4f0: e6a3 0091 | e603 869a | e513 00f9 | bf00 1feb | e583 0091 | e503 859a | e103 00f9 | 3f00 1feb + 0x000000011984a510: e103 00d1 | e103 819a | e801 0010 | 884f 01f9 | e803 0091 | 884b 01f9 + + 0x000000011984a528: ; {external_word} + 0x000000011984a528: c8df f7b0 | 08a9 5339 | 8806 0035 | 80c3 0a91 | e803 7eb2 | 89e3 0c91 | 28fd 9f88 + + 0x000000011984a544: ; {runtime_call Java_org_apache_comet_Native_executePlan} + 0x000000011984a544: 88c1 93d2 | a83f aaf2 | 2800 c0f2 | 0001 3fd6 | a800 80d2 | 883b 03b9 | bf3b 03d5 | 8803 0d91 + 0x000000011984a564: 08fd dfc8 | bf03 08eb | a803 0054 | 882b 43b9 | 6803 0035 | e803 7db2 | 89e3 0c91 | 28fd 9f88 + 0x000000011984a584: 88a3 4e39 | 1f09 0071 | c001 0054 + + 0x000000011984a590: ; {external_word} + 0x000000011984a590: c8df f7b0 | 08a9 5339 | 6805 0035 | 9f4b 01f9 | 9f4f 01f9 | 826f 40f9 | 5f80 00f9 | bf03 0091 + 0x000000011984a5b0: fd7b c1a8 | 8807 40f9 | 4800 00b5 | c003 5fd6 + + 0x000000011984a5c0: ; {runtime_call StubRoutines (1)} + 0x000000011984a5c0: b006 bd17 | a083 1ff8 + + 0x000000011984a5c8: ; {runtime_call SharedRuntime::reguard_yellow_pages()} + 0x000000011984a5c8: 8864 87d2 | 6820 a1f2 | 2800 c0f2 | 0001 3fd6 | a083 5ff8 | edff ff17 | a083 1ff8 | e003 1caa + 0x000000011984a5e8: ; {runtime_call JavaThread::check_special_condition_for_native_trans(JavaThread*)} + 0x000000011984a5e8: 0836 8ad2 | 2823 a1f2 | 2800 c0f2 | 0001 3fd6 | a083 5ff8 | dfff ff17 | e10b bda9 | e313 01a9 + 0x000000011984a608: e51b 02a9 + + 0x000000011984a60c: ; {metadata({method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native')} + 0x000000011984a60c: 01bf 98d2 | c1bf a6f2 | 2100 c0f2 | e003 1caa | e833 bfa9 | 082e 83d2 | 6820 a1f2 | 2800 c0f2 + 0x000000011984a62c: 0001 3fd6 | e833 c1a8 | e313 41a9 | e51b 42a9 | e10b c3a8 | bdff ff17 | a083 1ff8 + + 0x000000011984a648: ; {metadata({method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native')} + 0x000000011984a648: 01bf 98d2 | c1bf a6f2 | 2100 c0f2 | e003 1caa | e833 bfa9 | 0842 83d2 | 6820 a1f2 | 2800 c0f2 + 0x000000011984a668: 0001 3fd6 | e833 c1a8 | a083 5ff8 | caff ff17 +[/MachCode] + + +Compiled method (c1) 44945 18191 3 org.apache.comet.vector.NativeUtil::allocateArrowStructs (69 bytes) + total in heap [0x0000000112c2c890,0x0000000112c2dbc0] = 4912 + relocation [0x0000000112c2c9e8,0x0000000112c2cb50] = 360 + main code [0x0000000112c2cb80,0x0000000112c2d5c0] = 2624 + stub code [0x0000000112c2d5c0,0x0000000112c2d6f8] = 312 + oops [0x0000000112c2d6f8,0x0000000112c2d730] = 56 + metadata [0x0000000112c2d730,0x0000000112c2d7f8] = 200 + scopes data [0x0000000112c2d7f8,0x0000000112c2d9e8] = 496 + scopes pcs [0x0000000112c2d9e8,0x0000000112c2db88] = 416 + dependencies [0x0000000112c2db88,0x0000000112c2db90] = 8 + nul chk table [0x0000000112c2db90,0x0000000112c2dbc0] = 48 + +[Constant Pool (empty)] + +[MachCode] +[Entry Point] + # {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil' + # this: c_rarg1:c_rarg1 + = 'org/apache/comet/vector/NativeUtil' + # parm0: c_rarg2 = int + # [sp+0x120] (sp of caller) + 0x0000000112c2cb80: 2808 40b9 | 3f01 086b | c001 0054 + + 0x0000000112c2cb8c: ; {runtime_call ic_miss_stub} + 0x0000000112c2cb8c: 1d80 6e15 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 + 0x0000000112c2cbac: 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 +[Verified Entry Point] + 0x0000000112c2cbc0: 1f20 03d5 | e953 40d1 | 3f01 00f9 | ff83 04d1 | fd7b 11a9 | e163 00f9 + + 0x0000000112c2cbd8: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2cbd8: 00c3 90d2 | 60ca a6f2 | 2000 c0f2 | 03ac 40b9 | 6308 0011 | 03ac 00b9 | 6324 1f12 | 7f00 0071 + 0x0000000112c2cbf8: 203d 0054 | e2bb 00b9 + + 0x0000000112c2cc00: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2cc00: 00c3 90d2 | 60ca a6f2 | 2000 c0f2 | 0878 40f9 | 0805 0091 | 0878 00f9 + + 0x0000000112c2cc18: ; ImmutableOopMap {[192]=Oop } + ;*invokestatic $jacocoInit {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@0 + ; {static_call} + 0x0000000112c2cc18: 0a27 b595 | e073 00f9 | f3bb 40b9 + + 0x0000000112c2cc24: ; {metadata('org/apache/arrow/c/ArrowArray'[])} + 0x0000000112c2cc24: 0300 94d2 | 433c a0f2 | 0319 c0f2 | e403 00aa | 737e 40d3 | e503 13aa | e85f 40b2 | 7f02 08eb + 0x0000000112c2cc44: c23b 0054 | eb02 80d2 | 6b49 338b | 6bf1 7d92 | 8087 40f9 | 0b60 2b8b | 888f 40f9 | 7f01 08eb + 0x0000000112c2cc64: c83a 0054 | 8b87 00f9 | 6b01 00cb | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a4c 0129 | 6b41 00f1 + 0x0000000112c2cc84: c001 0054 | e803 0baa | 0a40 0091 | 0bfd 43d3 | 6821 00f1 | 4300 0054 + + 0x0000000112c2cc9c: ; {runtime_call StubRoutines (2)} + 0x0000000112c2cc9c: 892a 6e95 | 6b00 1036 | 5f7d 81a8 | 5f7d 81a8 | 4b00 0836 | 5f7d 81a8 | 4b00 0036 | 5f01 00f9 + 0x0000000112c2ccbc: bf3a 03d5 | e06b 00f9 | f3bb 40b9 + + 0x0000000112c2ccc8: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} + 0x0000000112c2ccc8: 038f 94d2 | 433c a0f2 | 0319 c0f2 | 737e 40d3 | e503 13aa | e85f 40b2 | 7f02 08eb | 0237 0054 + 0x0000000112c2cce8: eb02 80d2 | 6b49 338b | 6bf1 7d92 | 8087 40f9 | 0b60 2b8b | 888f 40f9 | 7f01 08eb | 0836 0054 + 0x0000000112c2cd08: 8b87 00f9 | 6b01 00cb | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a4c 0129 | 6b41 00f1 | c001 0054 + 0x0000000112c2cd28: e803 0baa | 0a40 0091 | 0bfd 43d3 | 6821 00f1 | 4300 0054 + + 0x0000000112c2cd3c: ; {runtime_call StubRoutines (2)} + 0x0000000112c2cd3c: 612a 6e95 | 6b00 1036 | 5f7d 81a8 | 5f7d 81a8 | 4b00 0836 | 5f7d 81a8 | 4b00 0036 | 5f01 00f9 + 0x0000000112c2cd5c: bf3a 03d5 | e06f 00f9 | e203 0032 + + 0x0000000112c2cd68: ; implicit exception: dispatches to 0x0000000112c2d3cc + 0x0000000112c2cd68: 810c 40b9 | 3f14 0071 | 0933 0054 | 8254 0039 + + 0x0000000112c2cd78: ; {oop(a 'java/lang/Class'{0x00000007039c8700} = 'scala/runtime/RichInt$')} + 0x0000000112c2cd78: 02e0 90d2 | 8273 a0f2 | e200 c0f2 | 4370 40b9 | 63f0 7dd3 + + 0x0000000112c2cd8c: ; {oop(a 'java/lang/Class'{0x0000000702fdfb20} = 'scala/Predef$')} + 0x0000000112c2cd8c: 0264 9fd2 | a25f a0f2 | e200 c0f2 | 4170 40b9 | 21f0 7dd3 + + 0x0000000112c2cda0: ; implicit exception: dispatches to 0x0000000112c2d3e8 + 0x0000000112c2cda0: 3f00 40f9 | e203 01aa + + 0x0000000112c2cda8: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2cda8: 05c3 90d2 | 65ca a6f2 | 2500 c0f2 | 4208 40b9 | 0219 c0f2 | a940 0491 | 2801 40f9 | 5f00 08eb + 0x0000000112c2cdc8: a100 0054 | a88c 40f9 | 0805 0091 | a88c 00f9 | 1c00 0014 | a980 0491 | 2801 40f9 | 5f00 08eb + 0x0000000112c2cde8: a100 0054 | a894 40f9 | 0805 0091 | a894 00f9 | 1400 0014 | a940 0491 | 2801 40f9 | c800 00b5 + 0x0000000112c2ce08: 2201 00f9 | e803 40b2 | a960 0491 | 2801 00f9 | 0c00 0014 | a980 0491 | 2801 40f9 | c800 00b5 + 0x0000000112c2ce28: 2201 00f9 | e803 40b2 | a9a0 0491 | 2801 00f9 | 0400 0014 | a880 40f9 | 0805 0091 | a880 00f9 + 0x0000000112c2ce48: 0200 a052 | e367 00f9 + + 0x0000000112c2ce50: ; ImmutableOopMap {[192]=Oop [200]=Oop [216]=Oop [208]=Oop [224]=Oop } + ;*invokevirtual intWrapper {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@27 (line 71) + ; {optimized virtual_call} + 0x0000000112c2ce50: 8010 b595 | e167 40f9 + + 0x0000000112c2ce58: ; implicit exception: dispatches to 0x0000000112c2d3ec + 0x0000000112c2ce58: 3f00 40f9 | e303 01aa + + 0x0000000112c2ce60: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2ce60: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 489c 40f9 | 0805 0091 | 489c 00f9 | e203 00aa | e3bb 40b9 + 0x0000000112c2ce80: ; ImmutableOopMap {[192]=Oop [216]=Oop [208]=Oop [224]=Oop } + ;*invokevirtual until$extension0 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@31 (line 71) + ; {optimized virtual_call} + 0x0000000112c2ce80: 44d0 b495 + + 0x0000000112c2ce84: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2ce84: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 4200 0691 | e163 40f9 | e303 01aa + + 0x0000000112c2ce9c: ; {metadata('org/apache/comet/vector/NativeUtil')} + 0x0000000112c2ce9c: 03f2 9ad2 | 433c a0f2 | 0319 c0f2 | 4904 40f9 | 6300 09ca | 68f4 7e92 | 4800 00b4 | 4304 00f9 + 0x0000000112c2cebc: e36b 40f9 | e403 03aa + + 0x0000000112c2cec4: ; {metadata('org/apache/arrow/c/ArrowArray'[])} + 0x0000000112c2cec4: 0400 94d2 | 443c a0f2 | 0419 c0f2 | 490c 40f9 | 8400 09ca | 88f4 7e92 | 0802 00b4 | e401 0837 + 0x0000000112c2cee4: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 8400 09ca | 490c 40f9 | 8400 09ca | 88f4 7e92 + 0x0000000112c2cf04: c800 00b4 | 490c 40f9 | 2901 7fb2 | 490c 00f9 | 0200 0014 | 440c 00f9 + + 0x0000000112c2cf1c: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2cf1c: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 48b8 40f9 | 0805 0091 | 48b8 00f9 + + 0x0000000112c2cf34: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c2cf34: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 44ac 40b9 | 8408 0011 | 44ac 00b9 | 844c 1f12 | 9f00 0071 + 0x0000000112c2cf54: e024 0054 + + 0x0000000112c2cf58: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2cf58: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 42c0 0891 + + 0x0000000112c2cf68: ; {oop(a 'java/lang/invoke/DirectMethodHandle$Constructor'{0x0000000712742d60})} + 0x0000000112c2cf68: 04ac 85d2 | 844e a2f2 | e400 c0f2 | e503 04aa | 8500 00b5 | 4900 40f9 | 2901 40b2 | 4900 00f9 + 0x0000000112c2cf88: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c2cf88: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 4894 40f9 | 0805 0091 | 4894 00f9 + + 0x0000000112c2cfa0: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2cfa0: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 45ac 40b9 | a508 0011 | 45ac 00b9 | a54c 1f12 | bf00 0071 + 0x0000000112c2cfc0: 8022 0054 + + 0x0000000112c2cfc4: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2cfc4: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0491 | e503 04aa | 8500 00b5 | 4904 40f9 | 2901 40b2 + 0x0000000112c2cfe4: 4904 00f9 + + 0x0000000112c2cfe8: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2cfe8: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4878 40f9 | 0805 0091 | 4878 00f9 + + 0x0000000112c2d000: ; {metadata(method data for {method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} + 0x0000000112c2d000: 0259 81d2 | c262 a5f2 | 2200 c0f2 | 45ac 40b9 | a508 0011 | 45ac 00b9 | a54c 1f12 | bf00 0071 + 0x0000000112c2d020: 8020 0054 + + 0x0000000112c2d024: ; {oop(a 'jdk/internal/misc/Unsafe'{0x0000000702af9ed0})} + 0x0000000112c2d024: 05da 93d2 | e555 a0f2 | e500 c0f2 | e203 05aa + + 0x0000000112c2d034: ; {metadata(method data for {method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} + 0x0000000112c2d034: 0659 81d2 | c662 a5f2 | 2600 c0f2 | c894 40f9 | 0805 0091 | c894 00f9 + + 0x0000000112c2d04c: ; {oop(a 'java/lang/Class'{0x0000000712742d88} = 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} + 0x0000000112c2d04c: 02b1 85d2 | 824e a2f2 | e200 c0f2 | e103 05aa | e483 0ea9 + + 0x0000000112c2d060: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [232]=Oop [208]=Oop [240]=Oop } + ;*invokevirtual allocateInstance {reexecute=0 rethrow=0 return_oop=0} + ; - java.lang.invoke.DirectMethodHandle::allocateInstance@12 (line 520) + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@1 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {optimized virtual_call} + 0x0000000112c2d060: 1c17 9295 + + 0x0000000112c2d064: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d064: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4240 0491 | e103 00aa | 8100 00b5 | 4900 40f9 | 2901 40b2 + 0x0000000112c2d084: 4900 00f9 + + 0x0000000112c2d088: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d088: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 42c0 0491 | e477 40f9 | e103 04aa | 8100 00b5 | 4904 40f9 + 0x0000000112c2d0a8: 2901 40b2 | 4904 00f9 + + 0x0000000112c2d0b0: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d0b0: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4890 40f9 | 0805 0091 | 4890 00f9 + + 0x0000000112c2d0c8: ; {metadata(method data for {method} {0x000000c8003a52f8} 'constructorMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} + 0x0000000112c2d0c8: 028d 81d2 | c262 a5f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 + 0x0000000112c2d0e8: 401b 0054 + + 0x0000000112c2d0ec: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d0ec: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0591 + + 0x0000000112c2d0fc: ; {oop(a 'java/lang/invoke/MemberName'{0x0000000712742df8} = {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} + 0x0000000112c2d0fc: 01bf 85d2 | 814e a2f2 | e100 c0f2 | 8100 00b5 | 4900 40f9 | 2901 40b2 | 4900 00f9 + + 0x0000000112c2d118: ; implicit exception: dispatches to 0x0000000112c2d470 + 0x0000000112c2d118: 1f00 40f9 | e203 00aa + + 0x0000000112c2d120: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d120: 0134 9dd2 | 81a2 a5f2 | 2100 c0f2 | 28c4 40f9 | 0805 0091 | 28c4 00f9 + + 0x0000000112c2d138: ; {metadata(method data for {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} + 0x0000000112c2d138: 020b 91d2 | 62ca a6f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 + 0x0000000112c2d158: e018 0054 | e203 00aa + + 0x0000000112c2d160: ; {metadata(method data for {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} + 0x0000000112c2d160: 010b 91d2 | 61ca a6f2 | 2100 c0f2 | 2878 40f9 | 0805 0091 | 2878 00f9 + + 0x0000000112c2d178: ; {metadata(method data for {method} {0x000000c800442c70} '' '()V' in 'java/lang/Object')} + 0x0000000112c2d178: 0224 95d2 | 0260 a5f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 + 0x0000000112c2d198: e017 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 | e163 40f9 | 28fc 43d3 | 080c 00b9 | 0200 01ca + 0x0000000112c2d1b8: 42fc 55d3 | 5f00 00f1 | 6118 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 | e36b 40f9 | 68fc 43d3 + 0x0000000112c2d1d8: 0810 00b9 | 0200 03ca | 42fc 55d3 | 5f00 00f1 | 6118 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 + 0x0000000112c2d1f8: e46f 40f9 | 88fc 43d3 | 0814 00b9 | 0200 04ca | 42fc 55d3 | 5f00 00f1 | 6118 0054 | bf3a 03d5 + 0x0000000112c2d218: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c2d218: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0691 | e103 00aa | 8100 00b5 | 4900 40f9 | 2901 40b2 + 0x0000000112c2d238: 4900 00f9 + + 0x0000000112c2d23c: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2d23c: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 4280 0691 | e103 00aa | a100 00b5 | 4900 40f9 | 2901 40b2 + 0x0000000112c2d25c: 4900 00f9 | 1600 0014 | 2108 40b9 | 0119 c0f2 | 4900 40f9 | 2100 09ca | 28f4 7e92 | 0802 00b4 + 0x0000000112c2d27c: e101 0837 | a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 2100 09ca | 4900 40f9 | 2100 09ca + 0x0000000112c2d29c: 28f4 7e92 | c800 00b4 | 4900 40f9 | 2901 7fb2 | 4900 00f9 | 0200 0014 | 4100 00f9 | e17b 40f9 + 0x0000000112c2d2bc: ; implicit exception: dispatches to 0x0000000112c2d52c + 0x0000000112c2d2bc: 3f00 40f9 | e203 01aa + + 0x0000000112c2d2c4: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2d2c4: 05c3 90d2 | 65ca a6f2 | 2500 c0f2 | a8d8 40f9 | 0805 0091 | a8d8 00f9 | e203 00aa + + 0x0000000112c2d2e0: ; ImmutableOopMap {[224]=Oop [208]=Oop [216]=Oop } + ;*invokevirtual foreach$mVc$sp {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@44 (line 71) + ; {optimized virtual_call} + 0x0000000112c2d2e0: 2c28 b495 | e303 0032 | e073 40f9 | 040c 40b9 | 9f18 0071 | e911 0054 | 0358 0039 + + 0x0000000112c2d2fc: ; {metadata('scala/Tuple2')} + 0x0000000112c2d2fc: 0356 92d2 | 8321 a0f2 | 0319 c0f2 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb | 8811 0054 + 0x0000000112c2d31c: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 | bf3a 03d5 + 0x0000000112c2d33c: e203 00aa + + 0x0000000112c2d340: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2d340: 03c3 90d2 | 63ca a6f2 | 2300 c0f2 | 68f4 40f9 | 0805 0091 | 68f4 00f9 | e20f 4da9 | e103 00aa + 0x0000000112c2d360: e07f 00f9 + + 0x0000000112c2d364: ; ImmutableOopMap {[248]=Oop [224]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@60 (line 78) + ; {optimized virtual_call} + 0x0000000112c2d364: bb23 9995 | e003 0032 | e173 40f9 | 220c 40b9 | 5f1c 0071 | c90e 0054 | 205c 0039 | e07f 40f9 + 0x0000000112c2d384: fd7b 51a9 | ff83 0491 + + 0x0000000112c2d38c: ; {poll_return} + 0x0000000112c2d38c: 88a3 41f9 | ff63 28eb | a80e 0054 | c003 5fd6 + + 0x0000000112c2d39c: ; {metadata({method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2d39c: 08bf 9ad2 | c8bf a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d3b4: ; ImmutableOopMap {c_rarg1=Oop [192]=Oop } + ;*synchronization entry + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@-1 + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d3b4: 5352 7195 | 11fe ff17 + + 0x0000000112c2d3bc: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop } + ;*anewarray {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@5 (line 68) + ; {runtime_call new_object_array Runtime1 stub} + 0x0000000112c2d3bc: 5143 7195 | 40fe ff17 + + 0x0000000112c2d3c4: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop } + ;*anewarray {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@11 (line 69) + ; {runtime_call new_object_array Runtime1 stub} + 0x0000000112c2d3c4: 4f43 7195 | 66fe ff17 + + 0x0000000112c2d3cc: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@19 (line 69) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c2d3cc: cd3e 7195 | a800 80d2 | e903 04aa + + 0x0000000112c2d3d8: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c2d3d8: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c2d3e4: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@19 (line 69) + 0x0000000112c2d3e4: c003 3fd6 + + 0x0000000112c2d3e8: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop c_rarg3=Oop c_rarg1=Oop } + ;*invokevirtual intWrapper {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@27 (line 71) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c2d3e8: c63e 7195 + + 0x0000000112c2d3ec: ; ImmutableOopMap {[192]=Oop [216]=Oop [208]=Oop [224]=Oop c_rarg1=Oop } + ;*invokevirtual until$extension0 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@31 (line 71) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c2d3ec: c53e 7195 + + 0x0000000112c2d3f0: ; {metadata({method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c2d3f0: 08f8 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d408: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop } + ;*synchronization entry + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d408: 3e52 7195 | d3fe ff17 + + 0x0000000112c2d410: ; {metadata({method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} + 0x0000000112c2d410: 08b4 8ed2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d428: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop } + ;*synchronization entry + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@-1 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d428: 3652 7195 | e6fe ff17 + + 0x0000000112c2d430: ; {metadata({method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} + 0x0000000112c2d430: 080a 89d2 | 4807 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d448: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop } + ;*synchronization entry + ; - java.lang.invoke.DirectMethodHandle::allocateInstance@-1 (line 519) + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@1 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d448: 2e52 7195 | f6fe ff17 + + 0x0000000112c2d450: ; {metadata({method} {0x000000c8003a52f8} 'constructorMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} + 0x0000000112c2d450: 085f 8ad2 | 4807 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d468: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } + ;*synchronization entry + ; - java.lang.invoke.DirectMethodHandle::constructorMethod@-1 (line 513) + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@7 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d468: 2652 7195 | 20ff ff17 + + 0x0000000112c2d470: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } + ;*invokestatic linkToSpecial {reexecute=0 rethrow=0 return_oop=0} + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c2d470: a43e 7195 + + 0x0000000112c2d474: ; {metadata({method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} + 0x0000000112c2d474: 08fa 82d2 | a8c0 a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d48c: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } + ;*synchronization entry + ; - org.apache.comet.vector.NativeUtil$$Lambda$3988/0x000000c801e28000::@-1 + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d48c: 1d52 7195 | 33ff ff17 + + 0x0000000112c2d494: ; {metadata({method} {0x000000c800442c70} '' '()V' in 'java/lang/Object')} + 0x0000000112c2d494: 088e 85d2 | 8808 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c2d4ac: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } + ;*synchronization entry + ; - java.lang.Object::@-1 (line 44) + ; - org.apache.comet.vector.NativeUtil$$Lambda$3988/0x000000c801e28000::@1 + ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c2d4ac: 1552 7195 | 3bff ff17 | 020c 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 + + 0x0000000112c2d4c4: ; {runtime_call g1_pre_barrier_slow} + 0x0000000112c2d4c4: 8f53 7195 | 38ff ff17 | c1e7 ffb4 | e003 00f9 + + 0x0000000112c2d4d4: ; {runtime_call g1_post_barrier_slow} + 0x0000000112c2d4d4: 4b54 7195 | 3bff ff17 | 0210 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 + + 0x0000000112c2d4ec: ; {runtime_call g1_pre_barrier_slow} + 0x0000000112c2d4ec: 8553 7195 | 38ff ff17 | c3e7 ffb4 | e003 00f9 + + 0x0000000112c2d4fc: ; {runtime_call g1_post_barrier_slow} + 0x0000000112c2d4fc: 4154 7195 | 3bff ff17 | 0214 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 + + 0x0000000112c2d514: ; {runtime_call g1_pre_barrier_slow} + 0x0000000112c2d514: 7b53 7195 | 38ff ff17 | c4e7 ffb4 | e003 00f9 + + 0x0000000112c2d524: ; {runtime_call g1_post_barrier_slow} + 0x0000000112c2d524: 3754 7195 | 3bff ff17 + + 0x0000000112c2d52c: ; ImmutableOopMap {[224]=Oop c_rarg0=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop [216]=Oop c_rarg1=Oop } + ;*invokevirtual foreach$mVc$sp {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@44 (line 71) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c2d52c: 753e 7195 | e807 7fb2 | e903 00aa + + 0x0000000112c2d538: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c2d538: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c2d544: ; ImmutableOopMap {[208]=Oop [216]=Oop c_rarg0=Oop [224]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@51 (line 71) + 0x0000000112c2d544: c003 3fd6 + + 0x0000000112c2d548: ; ImmutableOopMap {[208]=Oop [216]=Oop [224]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@52 (line 78) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c2d548: ae40 7195 | 7cff ff17 | e80b 40b2 | e903 01aa + + 0x0000000112c2d558: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c2d558: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c2d564: ; ImmutableOopMap {[248]=Oop c_rarg1=Oop [224]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@67 (line 78) + 0x0000000112c2d564: c003 3fd6 + + 0x0000000112c2d568: ; {internal_word} + 0x0000000112c2d568: 28f1 ff10 | 88af 01f9 + + 0x0000000112c2d570: ; {runtime_call SafepointBlob} + 0x0000000112c2d570: 2497 6e15 | 1f20 03d5 | 1f20 03d5 | 80eb 41f9 | 9feb 01f9 | 9fef 01f9 | fd7b 51a9 | ff83 0491 + 0x0000000112c2d590: ; {runtime_call unwind_exception Runtime1 stub} + 0x0000000112c2d590: 5c02 6f15 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 + 0x0000000112c2d5b0: 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 +[Stub Code] + 0x0000000112c2d5c0: ; {no_reloc} + 0x0000000112c2d5c0: df3f 03d5 + + 0x0000000112c2d5c4: ; {metadata(NULL)} + 0x0000000112c2d5c4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d5e0: ; {trampoline_stub} + 0x0000000112c2d5e0: 4800 0058 | 0001 1fd6 | 4068 9719 | 0100 0000 + + 0x0000000112c2d5f0: ; {static_stub} + 0x0000000112c2d5f0: df3f 03d5 + + 0x0000000112c2d5f4: ; {metadata(NULL)} + 0x0000000112c2d5f4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d610: ; {trampoline_stub} + 0x0000000112c2d610: 4800 0058 | 0001 1fd6 | 5010 9719 | 0100 0000 + + 0x0000000112c2d620: ; {static_stub} + 0x0000000112c2d620: df3f 03d5 + + 0x0000000112c2d624: ; {metadata(NULL)} + 0x0000000112c2d624: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d640: ; {trampoline_stub} + 0x0000000112c2d640: 4800 0058 | 0001 1fd6 | 900f 9619 | 0100 0000 + + 0x0000000112c2d650: ; {static_stub} + 0x0000000112c2d650: df3f 03d5 + + 0x0000000112c2d654: ; {metadata(NULL)} + 0x0000000112c2d654: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d670: ; {trampoline_stub} + 0x0000000112c2d670: 4800 0058 | 0001 1fd6 | d02c 0b19 | 0100 0000 + + 0x0000000112c2d680: ; {static_stub} + 0x0000000112c2d680: df3f 03d5 + + 0x0000000112c2d684: ; {metadata(NULL)} + 0x0000000112c2d684: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d6a0: ; {trampoline_stub} + 0x0000000112c2d6a0: 4800 0058 | 0001 1fd6 | 9073 9319 | 0100 0000 + + 0x0000000112c2d6b0: ; {static_stub} + 0x0000000112c2d6b0: df3f 03d5 + + 0x0000000112c2d6b4: ; {metadata(NULL)} + 0x0000000112c2d6b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c2d6d0: ; {trampoline_stub} + 0x0000000112c2d6d0: 4800 0058 | 0001 1fd6 | 5062 2719 | 0100 0000 +[Exception Handler] + 0x0000000112c2d6e0: ; {runtime_call handle_exception_from_callee Runtime1 stub} + 0x0000000112c2d6e0: 8845 7195 | c1d5 bbd4 | 3127 2a09 | 0100 0000 +[Deopt Handler Code] + 0x0000000112c2d6f0: 1e00 0010 + + 0x0000000112c2d6f4: ; {runtime_call DeoptimizationBlob} + 0x0000000112c2d6f4: 9397 6e15 +[/MachCode] + + +Compiled method (c1) 44954 18196 3 org.apache.comet.vector.NativeUtil::getNextBatch (296 bytes) + total in heap [0x0000000112c2f890,0x0000000112c32608] = 11640 + relocation [0x0000000112c2f9e8,0x0000000112c2fda8] = 960 + main code [0x0000000112c2fdc0,0x0000000112c31580] = 6080 + stub code [0x0000000112c31580,0x0000000112c31a48] = 1224 + oops [0x0000000112c31a48,0x0000000112c31aa0] = 88 + metadata [0x0000000112c31aa0,0x0000000112c31bf0] = 336 + scopes data [0x0000000112c31bf0,0x0000000112c32170] = 1408 + scopes pcs [0x0000000112c32170,0x0000000112c32590] = 1056 + dependencies [0x0000000112c32590,0x0000000112c32598] = 8 + nul chk table [0x0000000112c32598,0x0000000112c32608] = 112 + +[Constant Pool (empty)] + +[MachCode] +[Entry Point] + # {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil' + # this: c_rarg1:c_rarg1 + = 'org/apache/comet/vector/NativeUtil' + # parm0: c_rarg2 = int + # parm1: c_rarg3:c_rarg3 + = 'scala/Function2' + # [sp+0x150] (sp of caller) + 0x0000000112c2fdc0: 2808 40b9 | 3f01 086b | c001 0054 + + 0x0000000112c2fdcc: ; {runtime_call ic_miss_stub} + 0x0000000112c2fdcc: 8d73 6e15 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 + 0x0000000112c2fdec: 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 +[Verified Entry Point] + 0x0000000112c2fe00: 1f20 03d5 | e953 40d1 | 3f01 00f9 | ff43 05d1 | fd7b 14a9 | e347 00f9 + + 0x0000000112c2fe18: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fe18: 00b9 91d2 | 60ca a6f2 | 2000 c0f2 | 04ac 40b9 | 8408 0011 | 04ac 00b9 | 8424 1f12 | 9f00 0071 + 0x0000000112c2fe38: 60a9 0054 | e283 00b9 | e13f 00f9 + + 0x0000000112c2fe44: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fe44: 00b9 91d2 | 60ca a6f2 | 2000 c0f2 | 0878 40f9 | 0805 0091 | 0878 00f9 + + 0x0000000112c2fe5c: ; ImmutableOopMap {[120]=Oop [136]=Oop } + ;*invokestatic $jacocoInit {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@0 + ; {static_call} + 0x0000000112c2fe5c: 791a b595 | e04b 00f9 | e13f 40f9 + + 0x0000000112c2fe68: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fe68: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 + + 0x0000000112c2fe74: ; {metadata('org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fe74: 08f2 9ad2 | 483c a0f2 | 0819 c0f2 | 4940 0491 | 2801 00f9 | 488c 40f9 | 0805 0091 | 488c 00f9 + 0x0000000112c2fe94: e283 40b9 | e13f 40f9 + + 0x0000000112c2fe9c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop } + ;*invokevirtual allocateArrowStructs {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@7 (line 227) + ; {optimized virtual_call} + 0x0000000112c2fe9c: 49f3 ff97 | e103 1faa | 1f00 01eb + + 0x0000000112c2fea8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fea8: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 0827 80d2 | 0929 80d2 | 0201 899a | 2368 62f8 | 6304 0091 + 0x0000000112c2fec8: 2368 22f8 | 209f 0054 + + 0x0000000112c2fed0: ; implicit exception: dispatches to 0x0000000112c31384 + 0x0000000112c2fed0: 1f00 40f9 | e103 00aa + + 0x0000000112c2fed8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2fed8: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 49a0 0591 | 2801 40f9 | 3f00 08eb + 0x0000000112c2fef8: a100 0054 | 48b8 40f9 | 0805 0091 | 48b8 00f9 | 1c00 0014 | 49e0 0591 | 2801 40f9 | 3f00 08eb + 0x0000000112c2ff18: a100 0054 | 48c0 40f9 | 0805 0091 | 48c0 00f9 | 1400 0014 | 49a0 0591 | 2801 40f9 | c800 00b5 + 0x0000000112c2ff38: 2101 00f9 | e803 40b2 | 49c0 0591 | 2801 00f9 | 0c00 0014 | 49e0 0591 | 2801 40f9 | c800 00b5 + 0x0000000112c2ff58: 2101 00f9 | e803 40b2 | 4900 0691 | 2801 00f9 | 0400 0014 | 48ac 40f9 | 0805 0091 | 48ac 00f9 + 0x0000000112c2ff78: e103 00aa | e04f 00f9 | 0956 92d2 | 8921 a0f2 | 0919 c0f2 + + 0x0000000112c2ff8c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [152]=Oop } + ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@19 (line 227) + ; {virtual_call} + 0x0000000112c2ff8c: cd63 8d95 | 0001 00b5 + + 0x0000000112c2ff94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2ff94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6824 4639 | 0801 40b2 | 6824 0639 | 3700 0014 + + 0x0000000112c2ffb0: ; {metadata('org/apache/arrow/c/ArrowArray'[])} + 0x0000000112c2ffb0: 0200 94d2 | 423c a0f2 | 0219 c0f2 | 0308 40b9 | 0319 c0f2 | 6820 40f9 | 5f00 08eb | e104 0054 + 0x0000000112c2ffd0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c2ffd0: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0208 40b9 | 0219 c0f2 | 6980 0691 | 2801 40f9 | 5f00 08eb + 0x0000000112c2fff0: a100 0054 | 68d4 40f9 | 0805 0091 | 68d4 00f9 | 2200 0014 | 69c0 0691 | 2801 40f9 | 5f00 08eb + 0x0000000112c30010: a100 0054 | 68dc 40f9 | 0805 0091 | 68dc 00f9 | 1a00 0014 | 6980 0691 | 2801 40f9 | c800 00b5 + 0x0000000112c30030: 2201 00f9 | e803 40b2 | 69a0 0691 | 2801 00f9 | 1200 0014 | 69c0 0691 | 2801 40f9 | c800 00b5 + 0x0000000112c30050: 2201 00f9 | e803 40b2 | 69e0 0691 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c30068: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30068: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68c8 40f9 | 0805 00d1 | 68c8 00f9 | c204 0014 | 0100 0014 + 0x0000000112c30088: e14f 40f9 + + 0x0000000112c3008c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3008c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 4960 0791 | 2801 40f9 | 3f00 08eb + 0x0000000112c300ac: a100 0054 | 48f0 40f9 | 0805 0091 | 48f0 00f9 | 1c00 0014 | 49a0 0791 | 2801 40f9 | 3f00 08eb + 0x0000000112c300cc: a100 0054 | 48f8 40f9 | 0805 0091 | 48f8 00f9 | 1400 0014 | 4960 0791 | 2801 40f9 | c800 00b5 + 0x0000000112c300ec: 2101 00f9 | e803 40b2 | 4980 0791 | 2801 00f9 | 0c00 0014 | 49a0 0791 | 2801 40f9 | c800 00b5 + 0x0000000112c3010c: 2101 00f9 | e803 40b2 | 49c0 0791 | 2801 00f9 | 0400 0014 | 48e4 40f9 | 0805 0091 | 48e4 00f9 + 0x0000000112c3012c: e14f 40f9 | e053 00f9 | 0956 92d2 | 8921 a0f2 | 0919 c0f2 + + 0x0000000112c30140: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop } + ;*invokevirtual _2 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@29 (line 227) + ; {virtual_call} + 0x0000000112c30140: e02c 9995 | 0001 00b5 + + 0x0000000112c30148: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30148: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88e4 4739 | 0801 40b2 | 88e4 0739 | 3700 0014 + + 0x0000000112c30164: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} + 0x0000000112c30164: 018f 94d2 | 413c a0f2 | 0119 c0f2 | 0408 40b9 | 0419 c0f2 | 8820 40f9 | 3f00 08eb | e104 0054 + 0x0000000112c30184: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30184: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0108 40b9 | 0119 c0f2 | 8940 0891 | 2801 40f9 | 3f00 08eb + 0x0000000112c301a4: a100 0054 | 880c 41f9 | 0805 0091 | 880c 01f9 | 2200 0014 | 8980 0891 | 2801 40f9 | 3f00 08eb + 0x0000000112c301c4: a100 0054 | 8814 41f9 | 0805 0091 | 8814 01f9 | 1a00 0014 | 8940 0891 | 2801 40f9 | c800 00b5 + 0x0000000112c301e4: 2101 00f9 | e803 40b2 | 8960 0891 | 2801 00f9 | 1200 0014 | 8980 0891 | 2801 40f9 | c800 00b5 + 0x0000000112c30204: 2101 00f9 | e803 40b2 | 89a0 0891 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c3021c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3021c: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8800 41f9 | 0805 00d1 | 8800 01f9 | 5704 0014 | 0100 0014 + 0x0000000112c3023c: e103 00aa + + 0x0000000112c30240: ; {metadata('scala/Tuple2')} + 0x0000000112c30240: 0356 92d2 | 8321 a0f2 | 0319 c0f2 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb | e889 0054 + 0x0000000112c30260: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 | bf3a 03d5 + 0x0000000112c30280: e057 00f9 | e203 00aa + + 0x0000000112c30288: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30288: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 681c 41f9 | 0805 0091 | 681c 01f9 | e253 40f9 | e303 01aa + 0x0000000112c302a8: e103 00aa + + 0x0000000112c302ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@45 (line 227) + ; {optimized virtual_call} + 0x0000000112c302ac: e917 9995 + + 0x0000000112c302b0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c302b0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2024 41f9 | 0004 0091 | 2024 01f9 | e057 40f9 | e103 00aa + 0x0000000112c302d0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c302d0: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 2108 40b9 | 0119 c0f2 | 8960 0a91 | 2801 40f9 | 3f00 08eb + 0x0000000112c302f0: a100 0054 | 8850 41f9 | 0805 0091 | 8850 01f9 | 1c00 0014 | 89a0 0a91 | 2801 40f9 | 3f00 08eb + 0x0000000112c30310: a100 0054 | 8858 41f9 | 0805 0091 | 8858 01f9 | 1400 0014 | 8960 0a91 | 2801 40f9 | c800 00b5 + 0x0000000112c30330: 2101 00f9 | e803 40b2 | 8980 0a91 | 2801 00f9 | 0c00 0014 | 89a0 0a91 | 2801 40f9 | c800 00b5 + 0x0000000112c30350: 2101 00f9 | e803 40b2 | 89c0 0a91 | 2801 00f9 | 0400 0014 | 8844 41f9 | 0805 0091 | 8844 01f9 + 0x0000000112c30370: e103 00aa + + 0x0000000112c30374: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } + ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@74 (line 227) + ; {optimized virtual_call} + 0x0000000112c30374: d762 8d95 | 0001 00b5 + + 0x0000000112c3037c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3037c: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e4 4a39 | 0801 40b2 | 68e4 0a39 | 3700 0014 + + 0x0000000112c30398: ; {metadata('org/apache/arrow/c/ArrowArray'[])} + 0x0000000112c30398: 0200 94d2 | 423c a0f2 | 0219 c0f2 | 0308 40b9 | 0319 c0f2 | 6820 40f9 | 5f00 08eb | e104 0054 + 0x0000000112c303b8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c303b8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0208 40b9 | 0219 c0f2 | 6940 0b91 | 2801 40f9 | 5f00 08eb + 0x0000000112c303d8: a100 0054 | 686c 41f9 | 0805 0091 | 686c 01f9 | 2200 0014 | 6980 0b91 | 2801 40f9 | 5f00 08eb + 0x0000000112c303f8: a100 0054 | 6874 41f9 | 0805 0091 | 6874 01f9 | 1a00 0014 | 6940 0b91 | 2801 40f9 | c800 00b5 + 0x0000000112c30418: 2201 00f9 | e803 40b2 | 6960 0b91 | 2801 00f9 | 1200 0014 | 6980 0b91 | 2801 40f9 | c800 00b5 + 0x0000000112c30438: 2201 00f9 | e803 40b2 | 69a0 0b91 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c30450: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30450: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6860 41f9 | 0805 00d1 | 6860 01f9 | ce03 0014 | 0100 0014 + 0x0000000112c30470: e157 40f9 + + 0x0000000112c30474: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30474: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 4920 0c91 | 2801 40f9 | 3f00 08eb + 0x0000000112c30494: a100 0054 | 4888 41f9 | 0805 0091 | 4888 01f9 | 1c00 0014 | 4960 0c91 | 2801 40f9 | 3f00 08eb + 0x0000000112c304b4: a100 0054 | 4890 41f9 | 0805 0091 | 4890 01f9 | 1400 0014 | 4920 0c91 | 2801 40f9 | c800 00b5 + 0x0000000112c304d4: 2101 00f9 | e803 40b2 | 4940 0c91 | 2801 00f9 | 0c00 0014 | 4960 0c91 | 2801 40f9 | c800 00b5 + 0x0000000112c304f4: 2101 00f9 | e803 40b2 | 4980 0c91 | 2801 00f9 | 0400 0014 | 487c 41f9 | 0805 0091 | 487c 01f9 + 0x0000000112c30514: e157 40f9 | e05b 00f9 + + 0x0000000112c3051c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [176]=Oop } + ;*invokevirtual _2 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@84 (line 227) + ; {optimized virtual_call} + 0x0000000112c3051c: ed2b 9995 | 0001 00b5 + + 0x0000000112c30524: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30524: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88a4 4c39 | 0801 40b2 | 88a4 0c39 | 3700 0014 + + 0x0000000112c30540: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} + 0x0000000112c30540: 058f 94d2 | 453c a0f2 | 0519 c0f2 | 0408 40b9 | 0419 c0f2 | 8820 40f9 | bf00 08eb | e104 0054 + 0x0000000112c30560: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30560: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0508 40b9 | 0519 c0f2 | 8900 0d91 | 2801 40f9 | bf00 08eb + 0x0000000112c30580: a100 0054 | 88a4 41f9 | 0805 0091 | 88a4 01f9 | 2200 0014 | 8940 0d91 | 2801 40f9 | bf00 08eb + 0x0000000112c305a0: a100 0054 | 88ac 41f9 | 0805 0091 | 88ac 01f9 | 1a00 0014 | 8900 0d91 | 2801 40f9 | c800 00b5 + 0x0000000112c305c0: 2501 00f9 | e803 40b2 | 8920 0d91 | 2801 00f9 | 1200 0014 | 8940 0d91 | 2801 40f9 | c800 00b5 + 0x0000000112c305e0: 2501 00f9 | e803 40b2 | 8960 0d91 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c305f8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c305f8: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8898 41f9 | 0805 00d1 | 8898 01f9 | 6603 0014 | 0100 0014 + 0x0000000112c30618: e503 00aa | e303 0032 | e04b 40f9 + + 0x0000000112c30624: ; implicit exception: dispatches to 0x0000000112c313b0 + 0x0000000112c30624: 040c 40b9 | 9f7c 0071 | 496c 0054 | 03bc 0039 + + 0x0000000112c30634: ; {metadata('scala/collection/mutable/ArrayOps$ofRef')} + 0x0000000112c30634: 0303 94d2 | 4320 a0f2 | 0319 c0f2 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb | e86b 0054 + 0x0000000112c30654: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 + + 0x0000000112c3066c: ; {oop(a 'java/lang/Class'{0x0000000702fdfb20} = 'scala/Predef$')} + 0x0000000112c3066c: 0364 9fd2 | a35f a0f2 | e300 c0f2 | 6170 40b9 | 21f0 7dd3 + + 0x0000000112c30680: ; implicit exception: dispatches to 0x0000000112c313d4 + 0x0000000112c30680: 3f00 40f9 | e203 01aa + + 0x0000000112c30688: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30688: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88d0 41f9 | 0805 0091 | 88d0 01f9 | e25b 40f9 | e317 0ca9 + 0x0000000112c306a8: e05f 00f9 + + 0x0000000112c306ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } + ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@110 (line 229) + ; {optimized virtual_call} + 0x0000000112c306ac: a9f2 9a95 | e25f 40f9 + + 0x0000000112c306b4: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c306b4: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28ec 41f9 | 0805 0091 | 28ec 01f9 | e203 00aa | e15f 40f9 + 0x0000000112c306d4: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@113 (line 229) + ; {optimized virtual_call} + 0x0000000112c306d4: 6fe0 9b95 + + 0x0000000112c306d8: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c306d8: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2120 0791 + + 0x0000000112c306e8: ; {oop(a 'java/lang/invoke/BoundMethodHandle$Species_L'{0x000000071276b5a0})} + 0x0000000112c306e8: 00b4 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c30708: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30708: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28f4 41f9 | 0805 0091 | 28f4 01f9 + + 0x0000000112c30720: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30720: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 22ac 40b9 | 4208 0011 | 22ac 00b9 | 424c 1f12 | 5f00 0071 + 0x0000000112c30740: c064 0054 + + 0x0000000112c30744: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c30744: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 2100 0691 | 8000 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c30764: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30764: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2894 40f9 | 0805 0091 | 2894 00f9 + + 0x0000000112c3077c: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c3077c: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 20ac 40b9 | 0008 0011 | 20ac 00b9 | 004c 1f12 | 1f00 0071 + 0x0000000112c3079c: e062 0054 + + 0x0000000112c307a0: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c307a0: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2180 0591 + + 0x0000000112c307b0: ; {oop(a 'org/apache/comet/vector/NativeUtil$$Lambda$3991+0x000000c801e22000'{0x000000071276b5c0})} + 0x0000000112c307b0: 00b8 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c307d0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c307d0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 21e0 0f91 | e203 00aa | a200 00b5 | 2900 40f9 | 2901 40b2 + 0x0000000112c307f0: 2900 00f9 | 1700 0014 + + 0x0000000112c307f8: ; {metadata('org/apache/comet/vector/NativeUtil$$Lambda$3991+0x000000c801e22000')} + 0x0000000112c307f8: 0200 84d2 | 423c a0f2 | 0219 c0f2 | 2900 40f9 | 4200 09ca | 48f4 7e92 | 0802 00b4 | e201 0837 + 0x0000000112c30818: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 4200 09ca | 2900 40f9 | 4200 09ca | 48f4 7e92 + 0x0000000112c30838: c800 00b4 | 2900 40f9 | 2901 7fb2 | 2900 00f9 | 0200 0014 | 2200 00f9 + + 0x0000000112c30850: ; {oop(a 'java/lang/Class'{0x00000007039c32f8} = 'scala/Array$')} + 0x0000000112c30850: 025f 86d2 | 8273 a0f2 | e200 c0f2 | 4370 40b9 | 63f0 7dd3 + + 0x0000000112c30864: ; {oop(a 'java/lang/Class'{0x00000007039c31e8} = 'scala/reflect/ClassTag$')} + 0x0000000112c30864: 043d 86d2 | 8473 a0f2 | e400 c0f2 | 8170 40b9 | 21f0 7dd3 + + 0x0000000112c30878: ; implicit exception: dispatches to 0x0000000112c31418 + 0x0000000112c30878: 3f00 40f9 | e503 01aa + + 0x0000000112c30880: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30880: 06b9 91d2 | 66ca a6f2 | 2600 c0f2 | c804 42f9 | 0805 0091 | c804 02f9 | e213 0ea9 | e303 0da9 + 0x0000000112c308a0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [208]=Oop [224]=Oop [216]=Oop [232]=Oop } + ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@127 (line 229) + ; {optimized virtual_call} + 0x0000000112c308a0: ecd7 c595 | e36b 40f9 + + 0x0000000112c308a8: ; implicit exception: dispatches to 0x0000000112c3141c + 0x0000000112c308a8: 7f00 40f9 | e203 03aa + + 0x0000000112c308b0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c308b0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2820 42f9 | 0805 0091 | 2820 02f9 | e203 00aa | e103 03aa + 0x0000000112c308d0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [224]=Oop [216]=Oop [232]=Oop } + ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@130 (line 229) + ; {optimized virtual_call} + 0x0000000112c308d0: a020 bf95 | e25f 40f9 + + 0x0000000112c308d8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c308d8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 683c 42f9 | 0805 0091 | 683c 02f9 | e26f 40f9 | e303 00aa + 0x0000000112c308f8: e15f 40f9 + + 0x0000000112c308fc: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop } + ;*invokevirtual map {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@133 (line 229) + ; {optimized virtual_call} + 0x0000000112c308fc: 6523 bf95 | 0001 00b5 + + 0x0000000112c30904: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30904: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88a4 5239 | 0801 40b2 | 88a4 1239 | 3700 0014 + + 0x0000000112c30920: ; {metadata({type array long})} + 0x0000000112c30920: 0530 82d2 | 0500 a0f2 | 0519 c0f2 | 0408 40b9 | 0419 c0f2 | 881c 40f9 | bf00 08eb | e104 0054 + 0x0000000112c30940: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30940: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0508 40b9 | 0519 c0f2 | 8900 1391 | 2801 40f9 | bf00 08eb + 0x0000000112c30960: a100 0054 | 8864 42f9 | 0805 0091 | 8864 02f9 | 2200 0014 | 8940 1391 | 2801 40f9 | bf00 08eb + 0x0000000112c30980: a100 0054 | 886c 42f9 | 0805 0091 | 886c 02f9 | 1a00 0014 | 8900 1391 | 2801 40f9 | c800 00b5 + 0x0000000112c309a0: 2501 00f9 | e803 40b2 | 8920 1391 | 2801 00f9 | 1200 0014 | 8940 1391 | 2801 40f9 | c800 00b5 + 0x0000000112c309c0: 2501 00f9 | e803 40b2 | 8960 1391 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c309d8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c309d8: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8858 42f9 | 0805 00d1 | 8858 02f9 | 8c02 0014 | 0100 0014 + 0x0000000112c309f8: e503 00aa | e303 0032 | e04b 40f9 | 040c 40b9 | 9f80 0071 | e950 0054 | 03c0 0039 + + 0x0000000112c30a14: ; {metadata('scala/collection/mutable/ArrayOps$ofRef')} + 0x0000000112c30a14: 0303 94d2 | 4320 a0f2 | 0319 c0f2 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb | 8850 0054 + 0x0000000112c30a34: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 | e363 40f9 | 6170 40b9 + 0x0000000112c30a54: 21f0 7dd3 + + 0x0000000112c30a58: ; implicit exception: dispatches to 0x0000000112c31448 + 0x0000000112c30a58: 3f00 40f9 | e203 01aa + + 0x0000000112c30a60: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30a60: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6890 42f9 | 0805 0091 | 6890 02f9 | e267 40f9 | e017 0fa9 + 0x0000000112c30a80: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } + ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@159 (line 230) + ; {optimized virtual_call} + 0x0000000112c30a80: b4f1 9a95 | e27b 40f9 + + 0x0000000112c30a88: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30a88: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28ac 42f9 | 0805 0091 | 28ac 02f9 | e203 00aa | e17b 40f9 + 0x0000000112c30aa8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@162 (line 230) + ; {optimized virtual_call} + 0x0000000112c30aa8: 7adf 9b95 + + 0x0000000112c30aac: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30aac: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2120 0791 + + 0x0000000112c30abc: ; {oop(a 'java/lang/invoke/BoundMethodHandle$Species_L'{0x000000071276b5d0})} + 0x0000000112c30abc: 00ba 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c30adc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30adc: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28b4 42f9 | 0805 0091 | 28b4 02f9 + + 0x0000000112c30af4: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30af4: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 22ac 40b9 | 4208 0011 | 22ac 00b9 | 424c 1f12 | 5f00 0071 + 0x0000000112c30b14: c049 0054 + + 0x0000000112c30b18: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c30b18: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 2100 0691 | 8000 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c30b38: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30b38: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2894 40f9 | 0805 0091 | 2894 00f9 + + 0x0000000112c30b50: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c30b50: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 20ac 40b9 | 0008 0011 | 20ac 00b9 | 004c 1f12 | 1f00 0071 + 0x0000000112c30b70: e047 0054 + + 0x0000000112c30b74: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c30b74: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2180 0591 + + 0x0000000112c30b84: ; {oop(a 'org/apache/comet/vector/NativeUtil$$Lambda$3997+0x000000c801e1d000'{0x000000071276b5f0})} + 0x0000000112c30b84: 00be 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 + 0x0000000112c30ba4: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30ba4: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 21e0 1591 | e203 00aa | a200 00b5 | 2900 40f9 | 2901 40b2 + 0x0000000112c30bc4: 2900 00f9 | 1700 0014 + + 0x0000000112c30bcc: ; {metadata('org/apache/comet/vector/NativeUtil$$Lambda$3997+0x000000c801e1d000')} + 0x0000000112c30bcc: 0200 9ad2 | 223c a0f2 | 0219 c0f2 | 2900 40f9 | 4200 09ca | 48f4 7e92 | 0802 00b4 | e201 0837 + 0x0000000112c30bec: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 4200 09ca | 2900 40f9 | 4200 09ca | 48f4 7e92 + 0x0000000112c30c0c: c800 00b4 | 2900 40f9 | 2901 7fb2 | 2900 00f9 | 0200 0014 | 2200 00f9 | e273 40f9 | 4270 40b9 + 0x0000000112c30c2c: 42f0 7dd3 | e477 40f9 | 8170 40b9 | 21f0 7dd3 + + 0x0000000112c30c3c: ; implicit exception: dispatches to 0x0000000112c3148c + 0x0000000112c30c3c: 3f00 40f9 | e303 01aa + + 0x0000000112c30c44: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30c44: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88c4 42f9 | 0805 0091 | 88c4 02f9 | e203 10a9 + + 0x0000000112c30c60: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop [256]=Oop } + ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@176 (line 230) + ; {optimized virtual_call} + 0x0000000112c30c60: fcd6 c595 | e183 40f9 + + 0x0000000112c30c68: ; implicit exception: dispatches to 0x0000000112c31490 + 0x0000000112c30c68: 3f00 40f9 | e203 01aa + + 0x0000000112c30c70: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30c70: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e0 42f9 | 0805 0091 | 68e0 02f9 | e203 00aa + + 0x0000000112c30c8c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop } + ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@179 (line 230) + ; {optimized virtual_call} + 0x0000000112c30c8c: b11f bf95 | e27b 40f9 + + 0x0000000112c30c94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30c94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68fc 42f9 | 0805 0091 | 68fc 02f9 | e287 40f9 | e303 00aa + 0x0000000112c30cb4: e17b 40f9 + + 0x0000000112c30cb8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop } + ;*invokevirtual map {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@182 (line 230) + ; {optimized virtual_call} + 0x0000000112c30cb8: 7622 bf95 | 0001 00b5 + + 0x0000000112c30cc0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30cc0: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68a4 5839 | 0801 40b2 | 68a4 1839 | 3700 0014 + + 0x0000000112c30cdc: ; {metadata({type array long})} + 0x0000000112c30cdc: 0130 82d2 | 0100 a0f2 | 0119 c0f2 | 0308 40b9 | 0319 c0f2 | 681c 40f9 | 3f00 08eb | e104 0054 + 0x0000000112c30cfc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30cfc: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0108 40b9 | 0119 c0f2 | 6900 1991 | 2801 40f9 | 3f00 08eb + 0x0000000112c30d1c: a100 0054 | 6824 43f9 | 0805 0091 | 6824 03f9 | 2200 0014 | 6940 1991 | 2801 40f9 | 3f00 08eb + 0x0000000112c30d3c: a100 0054 | 682c 43f9 | 0805 0091 | 682c 03f9 | 1a00 0014 | 6900 1991 | 2801 40f9 | c800 00b5 + 0x0000000112c30d5c: 2101 00f9 | e803 40b2 | 6920 1991 | 2801 00f9 | 1200 0014 | 6940 1991 | 2801 40f9 | c800 00b5 + 0x0000000112c30d7c: 2101 00f9 | e803 40b2 | 6960 1991 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c30d94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30d94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6818 43f9 | 0805 00d1 | 6818 03f9 | ba01 0014 | 0100 0014 + 0x0000000112c30db4: e303 00aa | e203 0032 | e04b 40f9 | 010c 40b9 | 3f84 0071 | a936 0054 | 02c4 0039 | e147 40f9 + 0x0000000112c30dd4: ; implicit exception: dispatches to 0x0000000112c314b4 + 0x0000000112c30dd4: 3f00 40f9 | e203 01aa + + 0x0000000112c30ddc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30ddc: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 4208 40b9 | 0219 c0f2 | 89e0 1991 | 2801 40f9 | 5f00 08eb + 0x0000000112c30dfc: a100 0054 | 8840 43f9 | 0805 0091 | 8840 03f9 | 1c00 0014 | 8920 1a91 | 2801 40f9 | 5f00 08eb + 0x0000000112c30e1c: a100 0054 | 8848 43f9 | 0805 0091 | 8848 03f9 | 1400 0014 | 89e0 1991 | 2801 40f9 | c800 00b5 + 0x0000000112c30e3c: 2201 00f9 | e803 40b2 | 8900 1a91 | 2801 00f9 | 0c00 0014 | 8920 1a91 | 2801 40f9 | c800 00b5 + 0x0000000112c30e5c: 2201 00f9 | e803 40b2 | 8940 1a91 | 2801 00f9 | 0400 0014 | 8834 43f9 | 0805 0091 | 8834 03f9 + 0x0000000112c30e7c: e27f 40f9 | 09e4 94d2 | 49cd acf2 | 8901 c0f2 + + 0x0000000112c30e8c: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [144]=Oop } + ;*invokeinterface apply {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@201 (line 232) + ; {virtual_call} + 0x0000000112c30e8c: edfd 7995 + + 0x0000000112c30e90: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30e90: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2850 43f9 | 0805 0091 | 2850 03f9 | e103 00aa + + 0x0000000112c30eac: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [144]=Oop } + ;*invokestatic unboxToLong {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@206 (line 232) + ; {static_call} + 0x0000000112c30eac: 056f a095 | 0200 8092 | 5f00 00eb + + 0x0000000112c30eb8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30eb8: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 08d6 80d2 | 09d8 80d2 | 0311 899a | 4168 63f8 | 2104 0091 + 0x0000000112c30ed8: 4168 23f8 | 6102 0054 | e04b 40f9 + + 0x0000000112c30ee4: ; {oop(a 'java/lang/Class'{0x0000000702f28118} = 'scala/None$')} + 0x0000000112c30ee4: 0223 90d2 | 425e a0f2 | e200 c0f2 | 4270 40b9 | 42f0 7dd3 | e303 0032 | 010c 40b9 | 3f88 0071 + 0x0000000112c30f04: a92d 0054 | 03c8 0039 | e003 02aa | fd7b 54a9 | ff43 0591 + + 0x0000000112c30f18: ; {poll_return} + 0x0000000112c30f18: 88a3 41f9 | ff63 28eb | 882d 0054 | c003 5fd6 + + 0x0000000112c30f28: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30f28: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 4368 43f9 | 6304 0091 | 4368 03f9 | e08b 00f9 | e567 40f9 + 0x0000000112c30f48: e05b 40f9 | e13f 40f9 | e203 01aa + + 0x0000000112c30f54: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c30f54: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 + + 0x0000000112c30f60: ; {metadata('org/apache/comet/vector/NativeUtil')} + 0x0000000112c30f60: 08f2 9ad2 | 483c a0f2 | 0819 c0f2 | 69e0 1b91 | 2801 00f9 | 6880 43f9 | 0805 0091 | 6880 03f9 + 0x0000000112c30f80: e203 00aa | e303 05aa + + 0x0000000112c30f88: ; ImmutableOopMap {[144]=Oop } + ;*invokevirtual importVector {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@242 (line 239) + ; {optimized virtual_call} + 0x0000000112c30f88: ee29 6894 | e303 0032 | e54b 40f9 | a40c 40b9 | 9f8c 0071 | 092a 0054 | a3cc 0039 + + 0x0000000112c30fa4: ; {metadata('scala/Some')} + 0x0000000112c30fa4: 0300 90d2 | 2320 a0f2 | 0319 c0f2 | e08f 00f9 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb + 0x0000000112c30fc4: 8829 0054 | 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 + + 0x0000000112c30fe0: ; {metadata('org/apache/spark/sql/vectorized/ColumnarBatch')} + 0x0000000112c30fe0: 03f6 82d2 | e342 a0f2 | 0319 c0f2 | e097 00f9 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb + 0x0000000112c31000: e827 0054 | 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 + 0x0000000112c31020: bf3a 03d5 + + 0x0000000112c31024: ; {oop(a 'java/lang/Class'{0x00000007039c31e8} = 'scala/reflect/ClassTag$')} + 0x0000000112c31024: 023d 86d2 | 8273 a0f2 | e200 c0f2 | 4170 40b9 | 21f0 7dd3 + + 0x0000000112c31038: ; implicit exception: dispatches to 0x0000000112c31504 + 0x0000000112c31038: 3f00 40f9 | e203 01aa + + 0x0000000112c31040: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31040: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6890 43f9 | 0805 0091 | 6890 03f9 + + 0x0000000112c31058: ; {oop(a 'java/lang/Class'{0x0000000712525738} = 'org/apache/spark/sql/vectorized/ColumnVector')} + 0x0000000112c31058: 02e7 8ad2 | 424a a2f2 | e200 c0f2 | e093 00f9 + + 0x0000000112c31068: ; ImmutableOopMap {[280]=Oop [144]=Oop [296]=Oop [288]=Oop } + ;*invokevirtual apply {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@269 (line 240) + ; {optimized virtual_call} + 0x0000000112c31068: 8aa7 b395 | e18f 40f9 + + 0x0000000112c31070: ; implicit exception: dispatches to 0x0000000112c31508 + 0x0000000112c31070: 3f00 40f9 | e203 01aa + + 0x0000000112c31078: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31078: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 4208 40b9 | 0219 c0f2 | 69a0 1d91 | 2801 40f9 | 5f00 08eb + 0x0000000112c31098: a100 0054 | 68b8 43f9 | 0805 0091 | 68b8 03f9 | 1c00 0014 | 69e0 1d91 | 2801 40f9 | 5f00 08eb + 0x0000000112c310b8: a100 0054 | 68c0 43f9 | 0805 0091 | 68c0 03f9 | 1400 0014 | 69a0 1d91 | 2801 40f9 | c800 00b5 + 0x0000000112c310d8: 2201 00f9 | e803 40b2 | 69c0 1d91 | 2801 00f9 | 0c00 0014 | 69e0 1d91 | 2801 40f9 | c800 00b5 + 0x0000000112c310f8: 2201 00f9 | e803 40b2 | 6900 1e91 | 2801 00f9 | 0400 0014 | 68ac 43f9 | 0805 0091 | 68ac 03f9 + 0x0000000112c31118: e203 00aa | 09f4 95d2 | e921 a0f2 | 0919 c0f2 + + 0x0000000112c31128: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } + ;*invokeinterface toArray {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@272 (line 240) + ; {virtual_call} + 0x0000000112c31128: c6ba b995 | 0001 00b5 + + 0x0000000112c31130: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31130: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 4824 5e39 | 0801 40b2 | 4824 1e39 | 3700 0014 + + 0x0000000112c3114c: ; {metadata('org/apache/spark/sql/vectorized/ColumnVector'[])} + 0x0000000112c3114c: 01f7 9ad2 | a144 a0f2 | 0119 c0f2 | 0208 40b9 | 0219 c0f2 | 4820 40f9 | 3f00 08eb | e104 0054 + 0x0000000112c3116c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3116c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 0108 40b9 | 0119 c0f2 | 4980 1e91 | 2801 40f9 | 3f00 08eb + 0x0000000112c3118c: a100 0054 | 48d4 43f9 | 0805 0091 | 48d4 03f9 | 2200 0014 | 49c0 1e91 | 2801 40f9 | 3f00 08eb + 0x0000000112c311ac: a100 0054 | 48dc 43f9 | 0805 0091 | 48dc 03f9 | 1a00 0014 | 4980 1e91 | 2801 40f9 | c800 00b5 + 0x0000000112c311cc: 2101 00f9 | e803 40b2 | 49a0 1e91 | 2801 00f9 | 1200 0014 | 49c0 1e91 | 2801 40f9 | c800 00b5 + 0x0000000112c311ec: 2101 00f9 | e803 40b2 | 49e0 1e91 | 2801 00f9 | 0a00 0014 | 0900 0014 + + 0x0000000112c31204: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31204: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 48c8 43f9 | 0805 00d1 | 48c8 03f9 | bc00 0014 | 0100 0014 + 0x0000000112c31224: e203 00aa | e093 40f9 + + 0x0000000112c3122c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3122c: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e4 43f9 | 0805 0091 | 68e4 03f9 | e08b 40f9 | 007c 40d3 + 0x0000000112c3124c: e303 00aa | e193 40f9 + + 0x0000000112c31254: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@283 (line 240) + ; {optimized virtual_call} + 0x0000000112c31254: 1b73 dc97 | e097 40f9 + + 0x0000000112c3125c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c3125c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 48ec 43f9 | 0805 0091 | 48ec 03f9 | e207 52a9 + + 0x0000000112c31278: ; ImmutableOopMap {[144]=Oop [296]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@286 (line 240) + ; {optimized virtual_call} + 0x0000000112c31278: 16b7 9a95 | e003 0032 | e54b 40f9 | a30c 40b9 | 7f90 0071 | 4914 0054 | a0d0 0039 | e097 40f9 + 0x0000000112c31298: fd7b 54a9 | ff43 0591 + + 0x0000000112c312a0: ; {poll_return} + 0x0000000112c312a0: 88a3 41f9 | ff63 28eb | 2814 0054 | c003 5fd6 | e04f 00f9 | e54b 40f9 + + 0x0000000112c312b8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c312b8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6030 41f9 | 0004 0091 | 6030 01f9 | e24f 40f9 + + 0x0000000112c312d4: ; ImmutableOopMap {[144]=Oop c_rarg2=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) + ; {runtime_call load_mirror_patching Runtime1 stub} + 0x0000000112c312d4: 4b40 7195 | 68ec 4439 | 1f11 0071 + + 0x0000000112c312e0: ; implicit exception: dispatches to 0x0000000112c31538 + 0x0000000112c312e0: e112 0054 | 8087 40f9 | 0bc0 0091 | 888f 40f9 | 7f01 08eb | 4812 0054 | 8b87 00f9 | ea03 40b2 + 0x0000000112c31300: 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f7d 00a9 | 5f7d 01a9 | bf3a 03d5 | e103 00aa + 0x0000000112c31320: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31320: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 683c 41f9 | 0805 0091 | 683c 01f9 | e103 00aa | e09b 00f9 + 0x0000000112c31340: ; ImmutableOopMap {[144]=Oop [304]=Oop } + ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@60 (line 227) + ; {optimized virtual_call} + 0x0000000112c31340: f06e 6e95 | e003 0032 | e14b 40f9 + + 0x0000000112c3134c: ; implicit exception: dispatches to 0x0000000112c31544 + 0x0000000112c3134c: 220c 40b9 | 5f78 0071 | a90f 0054 | 20b8 0039 | e09b 40f9 | 8500 0014 + + 0x0000000112c31364: ; {metadata({method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31364: 08fa 9ad2 | c8bf a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c3137c: ; ImmutableOopMap {c_rarg1=Oop c_rarg3=Oop [136]=Oop } + ;*synchronization entry + ; - org.apache.comet.vector.NativeUtil::getNextBatch@-1 + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c3137c: 6142 7195 | affa ff17 + + 0x0000000112c31384: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop c_rarg0=Oop } + ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@19 (line 227) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31384: df2e 7195 | e803 00aa + + 0x0000000112c3138c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [152]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@22 (line 227) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c3138c: dd37 7195 | e803 00aa + + 0x0000000112c31394: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@32 (line 227) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c31394: db37 7195 + + 0x0000000112c31398: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop c_rarg1=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@37 (line 227) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c31398: 1a31 7195 | b9fb ff17 | e803 00aa + + 0x0000000112c313a4: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@77 (line 227) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c313a4: d737 7195 | e803 00aa + + 0x0000000112c313ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [176]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@87 (line 227) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c313ac: d537 7195 + + 0x0000000112c313b0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@97 (line 227) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c313b0: d42e 7195 | e813 40b2 | e903 00aa + + 0x0000000112c313bc: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c313bc: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c313c8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@97 (line 227) + 0x0000000112c313c8: c003 3fd6 + + 0x0000000112c313cc: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop [144]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@98 (line 229) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c313cc: 0d31 7195 | a7fc ff17 + + 0x0000000112c313d4: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop [144]=Oop c_rarg0=Oop c_rarg3=Oop c_rarg1=Oop } + ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@110 (line 229) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c313d4: cb2e 7195 + + 0x0000000112c313d8: ; {metadata({method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c313d8: 08ed 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c313f0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop c_rarg0=Oop } + ;*synchronization entry + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 + ; - org.apache.comet.vector.NativeUtil::getNextBatch@116 (line 229) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c313f0: 4442 7195 | d4fc ff17 + + 0x0000000112c313f8: ; {metadata({method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c313f8: 084d 80d2 | 0868 a5f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c31410: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } + ;*synchronization entry + ; - java.lang.invoke.LambdaForm$MH/0x000000c801000400::invoke@-1 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@4 + ; - org.apache.comet.vector.NativeUtil::getNextBatch@116 (line 229) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c31410: 3c42 7195 | e3fc ff17 + + 0x0000000112c31418: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop c_rarg0=Oop c_rarg2=Oop c_rarg3=Oop c_rarg4=Oop c_rarg1=Oop } + ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@127 (line 229) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31418: ba2e 7195 + + 0x0000000112c3141c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [224]=Oop [216]=Oop [232]=Oop c_rarg0=Oop c_rarg3=Oop } + ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@130 (line 229) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c3141c: b92e 7195 | e803 00aa + + 0x0000000112c31424: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@136 (line 229) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c31424: b737 7195 | e803 7bb2 | e903 00aa + + 0x0000000112c31430: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c31430: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c3143c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@146 (line 229) + 0x0000000112c3143c: c003 3fd6 + + 0x0000000112c31440: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop [144]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@147 (line 230) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c31440: f030 7195 | 82fd ff17 + + 0x0000000112c31448: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop [144]=Oop c_rarg0=Oop c_rarg1=Oop } + ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@159 (line 230) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31448: ae2e 7195 + + 0x0000000112c3144c: ; {metadata({method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} + 0x0000000112c3144c: 08ed 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c31464: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop c_rarg0=Oop } + ;*synchronization entry + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 + ; - org.apache.comet.vector.NativeUtil::getNextBatch@165 (line 230) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c31464: 2742 7195 | acfd ff17 + + 0x0000000112c3146c: ; {metadata({method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} + 0x0000000112c3146c: 084d 80d2 | 0868 a5f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 + + 0x0000000112c31484: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } + ;*synchronization entry + ; - java.lang.invoke.LambdaForm$MH/0x000000c801000400::invoke@-1 + ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@4 + ; - org.apache.comet.vector.NativeUtil::getNextBatch@165 (line 230) + ; {runtime_call counter_overflow Runtime1 stub} + 0x0000000112c31484: 1f42 7195 | bbfd ff17 + + 0x0000000112c3148c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop c_rarg0=Oop c_rarg2=Oop c_rarg1=Oop } + ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@176 (line 230) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c3148c: 9d2e 7195 + + 0x0000000112c31490: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop c_rarg0=Oop c_rarg1=Oop } + ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@179 (line 230) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31490: 9c2e 7195 | e803 00aa + + 0x0000000112c31498: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@185 (line 230) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c31498: 9a37 7195 | 2804 80d2 | e903 00aa + + 0x0000000112c314a4: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c314a4: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c314b0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop c_rarg3=Oop c_rarg0=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@195 (line 230) + 0x0000000112c314b0: c003 3fd6 + + 0x0000000112c314b4: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [248]=Oop c_rarg3=Oop c_rarg0=Oop [144]=Oop c_rarg1=Oop [136]=Oop } + ;*invokeinterface apply {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@201 (line 232) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c314b4: 932e 7195 | 4804 80d2 | e903 00aa + + 0x0000000112c314c0: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c314c0: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c314cc: ; ImmutableOopMap {c_rarg0=Oop [144]=Oop c_rarg2=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@232 (line 237) + 0x0000000112c314cc: c003 3fd6 + + 0x0000000112c314d0: ; {internal_word} + 0x0000000112c314d0: 48d2 ff10 | 88af 01f9 + + 0x0000000112c314d8: ; {runtime_call SafepointBlob} + 0x0000000112c314d8: 4a87 6e15 | 6804 80d2 | e903 05aa + + 0x0000000112c314e4: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c314e4: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c314f0: ; ImmutableOopMap {c_rarg0=Oop c_rarg5=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@252 (line 239) + 0x0000000112c314f0: c003 3fd6 + + 0x0000000112c314f4: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@253 (line 240) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c314f4: c330 7195 | bafe ff17 + + 0x0000000112c314fc: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop [296]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@257 (line 240) + ; {runtime_call fast_new_instance Runtime1 stub} + 0x0000000112c314fc: c130 7195 | c9fe ff17 + + 0x0000000112c31504: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop [296]=Oop c_rarg0=Oop c_rarg1=Oop } + ;*invokevirtual apply {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@269 (line 240) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31504: 7f2e 7195 + + 0x0000000112c31508: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop c_rarg0=Oop c_rarg1=Oop } + ;*invokeinterface toArray {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@272 (line 240) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31508: 7e2e 7195 | e803 00aa + + 0x0000000112c31510: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } + ;*checkcast {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@277 (line 240) + ; {runtime_call throw_class_cast_exception Runtime1 stub} + 0x0000000112c31510: 7c37 7195 | 8804 80d2 | e903 05aa + + 0x0000000112c3151c: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c3151c: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c31528: ; ImmutableOopMap {[296]=Oop c_rarg5=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@294 (line 240) + 0x0000000112c31528: c003 3fd6 + + 0x0000000112c3152c: ; {internal_word} + 0x0000000112c3152c: a8eb ff10 | 88af 01f9 + + 0x0000000112c31534: ; {runtime_call SafepointBlob} + 0x0000000112c31534: 3387 6e15 + + 0x0000000112c31538: ; ImmutableOopMap {c_rarg2=Oop [144]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31538: 722e 7195 + + 0x0000000112c3153c: ; ImmutableOopMap {c_rarg2=Oop [144]=Oop } + ;*new {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) + ; {runtime_call fast_new_instance_init_check Runtime1 stub} + 0x0000000112c3153c: 7131 7195 | 77ff ff17 + + 0x0000000112c31544: ; ImmutableOopMap {[304]=Oop c_rarg1=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@68 (line 227) + ; {runtime_call throw_null_pointer_exception Runtime1 stub} + 0x0000000112c31544: 6f2e 7195 | e80f 7fb2 | e903 01aa + + 0x0000000112c31550: ; {runtime_call throw_range_check_failed Runtime1 stub} + 0x0000000112c31550: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 + + 0x0000000112c3155c: ; ImmutableOopMap {[304]=Oop c_rarg1=Oop [144]=Oop } + ;*bastore {reexecute=0 rethrow=0 return_oop=0} + ; - org.apache.comet.vector.NativeUtil::getNextBatch@68 (line 227) + 0x0000000112c3155c: c003 3fd6 | 1f20 03d5 | 1f20 03d5 | 80eb 41f9 | 9feb 01f9 | 9fef 01f9 | fd7b 54a9 | ff43 0591 + 0x0000000112c3157c: ; {runtime_call unwind_exception Runtime1 stub} + 0x0000000112c3157c: 61f2 6e15 +[Stub Code] + 0x0000000112c31580: ; {no_reloc} + 0x0000000112c31580: df3f 03d5 + + 0x0000000112c31584: ; {metadata(NULL)} + 0x0000000112c31584: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c315a0: ; {trampoline_stub} + 0x0000000112c315a0: 4800 0058 | 0001 1fd6 | 4068 9719 | 0100 0000 + + 0x0000000112c315b0: ; {static_stub} + 0x0000000112c315b0: df3f 03d5 + + 0x0000000112c315b4: ; {metadata(NULL)} + 0x0000000112c315b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c315d0: ; {trampoline_stub} + 0x0000000112c315d0: 4800 0058 | 0001 1fd6 | c0cb c212 | 0100 0000 + + 0x0000000112c315e0: ; {static_stub} + 0x0000000112c315e0: df3f 03d5 + + 0x0000000112c315e4: ; {metadata(NULL)} + 0x0000000112c315e4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31600: ; {trampoline_stub} + 0x0000000112c31600: 4800 0058 | 0001 1fd6 | c08e f818 | 0100 0000 + + 0x0000000112c31610: ; {static_stub} + 0x0000000112c31610: df3f 03d5 + + 0x0000000112c31614: ; {metadata(NULL)} + 0x0000000112c31614: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31630: ; {trampoline_stub} + 0x0000000112c31630: 4800 0058 | 0001 1fd6 | c0b4 2719 | 0100 0000 + + 0x0000000112c31640: ; {static_stub} + 0x0000000112c31640: df3f 03d5 + + 0x0000000112c31644: ; {metadata(NULL)} + 0x0000000112c31644: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31660: ; {trampoline_stub} + 0x0000000112c31660: 4800 0058 | 0001 1fd6 | 5062 2719 | 0100 0000 + + 0x0000000112c31670: ; {static_stub} + 0x0000000112c31670: df3f 03d5 + + 0x0000000112c31674: ; {metadata(NULL)} + 0x0000000112c31674: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31690: ; {trampoline_stub} + 0x0000000112c31690: 4800 0058 | 0001 1fd6 | d08e f818 | 0100 0000 + + 0x0000000112c316a0: ; {static_stub} + 0x0000000112c316a0: df3f 03d5 + + 0x0000000112c316a4: ; {metadata(NULL)} + 0x0000000112c316a4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c316c0: ; {trampoline_stub} + 0x0000000112c316c0: 4800 0058 | 0001 1fd6 | d0b4 2719 | 0100 0000 + + 0x0000000112c316d0: ; {static_stub} + 0x0000000112c316d0: df3f 03d5 + + 0x0000000112c316d4: ; {metadata(NULL)} + 0x0000000112c316d4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c316f0: ; {trampoline_stub} + 0x0000000112c316f0: 4800 0058 | 0001 1fd6 | 50d1 2e19 | 0100 0000 + + 0x0000000112c31700: ; {static_stub} + 0x0000000112c31700: df3f 03d5 + + 0x0000000112c31704: ; {metadata(NULL)} + 0x0000000112c31704: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31720: ; {trampoline_stub} + 0x0000000112c31720: 4800 0058 | 0001 1fd6 | 9088 3219 | 0100 0000 + + 0x0000000112c31730: ; {static_stub} + 0x0000000112c31730: df3f 03d5 + + 0x0000000112c31734: ; {metadata(NULL)} + 0x0000000112c31734: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31750: ; {trampoline_stub} + 0x0000000112c31750: 4800 0058 | 0001 1fd6 | 5068 da19 | 0100 0000 + + 0x0000000112c31760: ; {static_stub} + 0x0000000112c31760: df3f 03d5 + + 0x0000000112c31764: ; {metadata(NULL)} + 0x0000000112c31764: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31780: ; {trampoline_stub} + 0x0000000112c31780: 4800 0058 | 0001 1fd6 | 508b bf19 | 0100 0000 + + 0x0000000112c31790: ; {static_stub} + 0x0000000112c31790: df3f 03d5 + + 0x0000000112c31794: ; {metadata(NULL)} + 0x0000000112c31794: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c317b0: ; {trampoline_stub} + 0x0000000112c317b0: 4800 0058 | 0001 1fd6 | 9096 bf19 | 0100 0000 + + 0x0000000112c317c0: ; {static_stub} + 0x0000000112c317c0: df3f 03d5 + + 0x0000000112c317c4: ; {metadata(NULL)} + 0x0000000112c317c4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c317e0: ; {trampoline_stub} + 0x0000000112c317e0: 4800 0058 | 0001 1fd6 | 50d1 2e19 | 0100 0000 + + 0x0000000112c317f0: ; {static_stub} + 0x0000000112c317f0: df3f 03d5 + + 0x0000000112c317f4: ; {metadata(NULL)} + 0x0000000112c317f4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31810: ; {trampoline_stub} + 0x0000000112c31810: 4800 0058 | 0001 1fd6 | 9088 3219 | 0100 0000 + + 0x0000000112c31820: ; {static_stub} + 0x0000000112c31820: df3f 03d5 + + 0x0000000112c31824: ; {metadata(NULL)} + 0x0000000112c31824: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31840: ; {trampoline_stub} + 0x0000000112c31840: 4800 0058 | 0001 1fd6 | 5068 da19 | 0100 0000 + + 0x0000000112c31850: ; {static_stub} + 0x0000000112c31850: df3f 03d5 + + 0x0000000112c31854: ; {metadata(NULL)} + 0x0000000112c31854: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31870: ; {trampoline_stub} + 0x0000000112c31870: 4800 0058 | 0001 1fd6 | 508b bf19 | 0100 0000 + + 0x0000000112c31880: ; {static_stub} + 0x0000000112c31880: df3f 03d5 + + 0x0000000112c31884: ; {metadata(NULL)} + 0x0000000112c31884: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c318a0: ; {trampoline_stub} + 0x0000000112c318a0: 4800 0058 | 0001 1fd6 | 9096 bf19 | 0100 0000 + + 0x0000000112c318b0: ; {static_stub} + 0x0000000112c318b0: df3f 03d5 + + 0x0000000112c318b4: ; {metadata(NULL)} + 0x0000000112c318b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c318d0: ; {trampoline_stub} + 0x0000000112c318d0: 4800 0058 | 0001 1fd6 | 4006 ab18 | 0100 0000 + + 0x0000000112c318e0: ; {static_stub} + 0x0000000112c318e0: df3f 03d5 + + 0x0000000112c318e4: ; {metadata(NULL)} + 0x0000000112c318e4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31900: ; {trampoline_stub} + 0x0000000112c31900: 4800 0058 | 0001 1fd6 | c0ca 4419 | 0100 0000 + + 0x0000000112c31910: ; {static_stub} + 0x0000000112c31910: df3f 03d5 + + 0x0000000112c31914: ; {metadata({method} {0x0000000135fed960} 'importVector' '([Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)Lscala/collection/Seq;' in 'org/apache/comet/vector/NativeUtil')} + 0x0000000112c31914: 0c2c 9bd2 | ccbf a6f2 | 2c00 c0f2 | 88e7 92d2 | 880f a3f2 | 2800 c0f2 | 0001 1fd6 + + 0x0000000112c31930: ; {trampoline_stub} + 0x0000000112c31930: 4800 0058 | 0001 1fd6 | 40b7 6314 | 0100 0000 + + 0x0000000112c31940: ; {static_stub} + 0x0000000112c31940: df3f 03d5 + + 0x0000000112c31944: ; {metadata(NULL)} + 0x0000000112c31944: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31960: ; {trampoline_stub} + 0x0000000112c31960: 4800 0058 | 0001 1fd6 | 90ae 9119 | 0100 0000 + + 0x0000000112c31970: ; {static_stub} + 0x0000000112c31970: df3f 03d5 + + 0x0000000112c31974: ; {metadata(NULL)} + 0x0000000112c31974: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31990: ; {trampoline_stub} + 0x0000000112c31990: 4800 0058 | 0001 1fd6 | 40fc a919 | 0100 0000 + + 0x0000000112c319a0: ; {static_stub} + 0x0000000112c319a0: df3f 03d5 + + 0x0000000112c319a4: ; {metadata({method} {0x0000000135f02840} '' '([Lorg/apache/spark/sql/vectorized/ColumnVector;I)V' in 'org/apache/spark/sql/vectorized/ColumnarBatch')} + 0x0000000112c319a4: 0c08 85d2 | 0cbe a6f2 | 2c00 c0f2 | 8887 8fd2 | 880f a3f2 | 2800 c0f2 | 0001 1fd6 + + 0x0000000112c319c0: ; {trampoline_stub} + 0x0000000112c319c0: 4800 0058 | 0001 1fd6 | c0de 3412 | 0100 0000 + + 0x0000000112c319d0: ; {static_stub} + 0x0000000112c319d0: df3f 03d5 + + 0x0000000112c319d4: ; {metadata(NULL)} + 0x0000000112c319d4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c319f0: ; {trampoline_stub} + 0x0000000112c319f0: 4800 0058 | 0001 1fd6 | d0ee 2d19 | 0100 0000 + + 0x0000000112c31a00: ; {static_stub} + 0x0000000112c31a00: df3f 03d5 + + 0x0000000112c31a04: ; {metadata(NULL)} + 0x0000000112c31a04: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 + + 0x0000000112c31a20: ; {trampoline_stub} + 0x0000000112c31a20: 4800 0058 | 0001 1fd6 | 00cf 7c18 | 0100 0000 +[Exception Handler] + 0x0000000112c31a30: ; {runtime_call handle_exception_from_callee Runtime1 stub} + 0x0000000112c31a30: b434 7195 | c1d5 bbd4 | 3127 2a09 | 0100 0000 +[Deopt Handler Code] + 0x0000000112c31a40: 1e00 0010 + + 0x0000000112c31a44: ; {runtime_call DeoptimizationBlob} + 0x0000000112c31a44: bf86 6e15 +[/MachCode] + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x0000000c638ca180, length=286, elements={ +0x00000001033f4320, 0x00000001034028a0, 0x0000000c688a4000, 0x0000000c688a4c00, +0x0000000c688a5200, 0x0000000c688a5800, 0x0000000c688a5e00, 0x0000000c688a6400, +0x0000000c688a6a00, 0x0000000c688a7000, 0x0000000c6890a400, 0x0000000c6862c600, +0x0000000c66c26a00, 0x0000000c687c5800, 0x0000000c687c5200, 0x0000000c687c4c00, +0x0000000c687c4600, 0x0000000c687c5e00, 0x0000000c66cbc000, 0x0000000c66cbc600, +0x0000000c66cbcc00, 0x0000000c66cbd200, 0x0000000c66cbd800, 0x0000000c66cbde00, +0x0000000c66cbe400, 0x0000000c66cbea00, 0x0000000c66ccd200, 0x0000000c66ccf000, +0x0000000c66ccf600, 0x0000000c66d1c600, 0x0000000c66d1cc00, 0x0000000c66d1d800, +0x0000000c66d1de00, 0x0000000c66d1e400, 0x0000000c66d1f000, 0x0000000c66d1f600, +0x0000000c66d68c00, 0x0000000c66d69200, 0x0000000c66d69e00, 0x0000000c66d6aa00, +0x0000000c66d6b000, 0x0000000c66d6b600, 0x0000000c66dd8000, 0x0000000c66de4600, +0x0000000c66de5e00, 0x0000000c66ec1800, 0x0000000c66530c00, 0x0000000c67a99800, +0x0000000c67a99e00, 0x0000000c67a9a400, 0x0000000c67a9aa00, 0x0000000c67a9b000, +0x0000000c676b0600, 0x0000000c676b2400, 0x0000000c67aab600, 0x0000000c67aaaa00, +0x0000000c67130600, 0x0000000c67130c00, 0x0000000c67131200, 0x0000000c67131800, +0x0000000c65d64600, 0x0000000c65d64c00, 0x0000000c65d65200, 0x0000000c65d65800, +0x0000000c65d65e00, 0x0000000c65e5a400, 0x0000000c65e59e00, 0x0000000c65e5aa00, +0x0000000c65e5b000, 0x0000000c65e5b600, 0x0000000c64688000, 0x0000000c6862f000, +0x0000000c677e6a00, 0x0000000c677e4000, 0x0000000c677e4c00, 0x0000000c67132a00, +0x0000000c654ad200, 0x0000000c654ad800, 0x0000000c654ade00, 0x0000000c654aea00, +0x0000000c654af000, 0x0000000c654af600, 0x0000000c688a7600, 0x0000000c677e5800, +0x0000000c677e5e00, 0x0000000c677e7000, 0x0000000c677e7600, 0x0000000c67aaa400, +0x0000000c6890b600, 0x0000000c68908000, 0x0000000c65e59200, 0x0000000c646f8c00, +0x0000000c646f8600, 0x0000000c640f0000, 0x0000000c640f0600, 0x0000000c640f0c00, +0x0000000c640f1200, 0x0000000c640f1800, 0x0000000c640f1e00, 0x0000000c640f2400, +0x0000000c640f2a00, 0x0000000c640f3000, 0x0000000c640f3600, 0x0000000c667b4000, +0x0000000c667b4600, 0x0000000c667b4c00, 0x0000000c667b5200, 0x0000000c667b5800, +0x0000000c667b5e00, 0x0000000c667b6400, 0x0000000c667b6a00, 0x0000000c667b7000, +0x0000000c667b7600, 0x0000000c6557c000, 0x0000000c6557c600, 0x0000000c6557cc00, +0x0000000c6557d200, 0x0000000c6557d800, 0x0000000c6557de00, 0x0000000c6557e400, +0x0000000c6557ea00, 0x0000000c6557f000, 0x0000000c6557f600, 0x0000000c640fc000, +0x0000000c640fc600, 0x0000000c640fcc00, 0x0000000c640fd200, 0x0000000c640fd800, +0x0000000c640ff000, 0x0000000c655bd200, 0x0000000c655be400, 0x0000000c655bea00, +0x0000000c655bf000, 0x0000000c655bf600, 0x0000000c66de6a00, 0x0000000c655e4000, +0x0000000c655e4600, 0x0000000c655e4c00, 0x0000000c655e5200, 0x0000000c655e5800, +0x0000000c655e5e00, 0x0000000c655e6400, 0x0000000c655e6a00, 0x0000000c655e7000, +0x0000000c655e7600, 0x0000000c64724000, 0x0000000c64724600, 0x0000000c64724c00, +0x0000000c64725200, 0x0000000c64725800, 0x0000000c64725e00, 0x0000000c64726400, +0x0000000c64726a00, 0x0000000c64727000, 0x0000000c64727600, 0x0000000c6412c000, +0x0000000c6412c600, 0x0000000c6412cc00, 0x0000000c6412d200, 0x0000000c6412d800, +0x0000000c6412e400, 0x0000000c6412de00, 0x0000000c6412ea00, 0x0000000c6412f000, +0x0000000c64134600, 0x0000000c64134000, 0x0000000c64134c00, 0x0000000c64135200, +0x0000000c64135800, 0x0000000c64135e00, 0x0000000c64136400, 0x0000000c64136a00, +0x0000000c64137000, 0x0000000c64137600, 0x0000000c655ec600, 0x0000000c655ec000, +0x0000000c655ecc00, 0x0000000c655ed200, 0x0000000c655ed800, 0x0000000c655ede00, +0x0000000c655ee400, 0x0000000c655eea00, 0x0000000c655ef000, 0x0000000c655ef600, +0x0000000c655f0600, 0x0000000c655f0000, 0x0000000c655f1e00, 0x0000000c655f3600, +0x0000000c64738600, 0x0000000c6473b600, 0x0000000c6473b000, 0x0000000c655f2400, +0x0000000c6412f600, 0x0000000c64754000, 0x0000000c64754600, 0x0000000c64754c00, +0x0000000c64755200, 0x0000000c64755800, 0x0000000c64755e00, 0x0000000c64756400, +0x0000000c64756a00, 0x0000000c64757000, 0x0000000c64757600, 0x0000000c64758000, +0x0000000c64758600, 0x0000000c64758c00, 0x0000000c64759200, 0x0000000c64759800, +0x0000000c64759e00, 0x0000000c6475a400, 0x0000000c6475aa00, 0x0000000c6475b000, +0x0000000c6475b600, 0x0000000c65600000, 0x0000000c65600600, 0x0000000c65601200, +0x0000000c65601800, 0x0000000c65601e00, 0x0000000c65602400, 0x0000000c65602a00, +0x0000000c65603000, 0x0000000c65603600, 0x0000000c65608000, 0x0000000c65608600, +0x0000000c65608c00, 0x0000000c65609200, 0x0000000c65609800, 0x0000000c65609e00, +0x0000000c6560a400, 0x0000000c6560aa00, 0x0000000c6560b000, 0x0000000c6560b600, +0x0000000c64170000, 0x0000000c64170600, 0x0000000c64170c00, 0x0000000c64171200, +0x0000000c64171800, 0x0000000c64171e00, 0x0000000c64172400, 0x0000000c64172a00, +0x0000000c64173000, 0x0000000c64173600, 0x0000000c64768000, 0x0000000c64768600, +0x0000000c64769200, 0x0000000c64769800, 0x0000000c641a0600, 0x0000000c641a1800, +0x0000000c641a1e00, 0x0000000c641a2400, 0x0000000c641a2a00, 0x0000000c641a3000, +0x0000000c641a3600, 0x0000000c646fb600, 0x0000000c65dbde00, 0x0000000c646e1200, +0x0000000c66531800, 0x0000000c66531200, 0x0000000c641a8000, 0x0000000c641a8600, +0x0000000c641a8c00, 0x0000000c641a9200, 0x0000000c641a9800, 0x0000000c641a9e00, +0x0000000c641aa400, 0x0000000c641aaa00, 0x0000000c641ab000, 0x0000000c641ab600, +0x0000000c6477c600, 0x0000000c6477d200, 0x0000000c6477d800, 0x0000000c6477e400, +0x0000000c6477ea00, 0x0000000c6477f600, 0x0000000c64200000, 0x0000000c64200600, +0x0000000c64200c00, 0x0000000c64201200, 0x0000000c64201800, 0x0000000c64201e00, +0x0000000c64202400, 0x0000000c64202a00, 0x0000000c64203000, 0x0000000c64203600, +0x0000000c64204000, 0x0000000c64289e00 +} + +Java Threads: ( => current thread ) + 0x00000001033f4320 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5891, stack(0x000000016d34c000,0x000000016d74f000)] + 0x00000001034028a0 JavaThread "Reference Handler" daemon [_thread_blocked, id=30723, stack(0x000000016e3a0000,0x000000016e7a3000)] + 0x0000000c688a4000 JavaThread "Finalizer" daemon [_thread_blocked, id=22531, stack(0x000000016e7ac000,0x000000016ebaf000)] + 0x0000000c688a4c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=28675, stack(0x000000016ecd0000,0x000000016f0d3000)] + 0x0000000c688a5200 JavaThread "Service Thread" daemon [_thread_blocked, id=24579, stack(0x000000016f0dc000,0x000000016f4df000)] + 0x0000000c688a5800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=28419, stack(0x000000016f4e8000,0x000000016f8eb000)] + 0x0000000c688a5e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=27907, stack(0x000000016f8f4000,0x000000016faf7000)] + 0x0000000c688a6400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=25347, stack(0x000000016fb00000,0x000000016fd03000)] + 0x0000000c688a6a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27651, stack(0x000000016fd0c000,0x000000017010f000)] + 0x0000000c688a7000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27395, stack(0x0000000170118000,0x000000017051b000)] + 0x0000000c6890a400 JavaThread "Notification Thread" daemon [_thread_blocked, id=42499, stack(0x0000000170f60000,0x0000000171363000)] + 0x0000000c6862c600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=65283, stack(0x0000000173e68000,0x000000017426b000)] + 0x0000000c66c26a00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=47107, stack(0x0000000174cb0000,0x00000001750b3000)] + 0x0000000c687c5800 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=63747, stack(0x00000001750bc000,0x00000001754bf000)] + 0x0000000c687c5200 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=63235, stack(0x00000001754c8000,0x00000001758cb000)] + 0x0000000c687c4c00 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=62979, stack(0x00000001758d4000,0x0000000175cd7000)] + 0x0000000c687c4600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=47875, stack(0x0000000175ce0000,0x00000001760e3000)] + 0x0000000c687c5e00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=48131, stack(0x00000001760ec000,0x00000001764ef000)] + 0x0000000c66cbc000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=62211, stack(0x00000001764f8000,0x00000001768fb000)] + 0x0000000c66cbc600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=48899, stack(0x0000000176904000,0x0000000176d07000)] + 0x0000000c66cbcc00 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=61699, stack(0x0000000176d10000,0x0000000177113000)] + 0x0000000c66cbd200 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=49411, stack(0x000000017711c000,0x000000017751f000)] + 0x0000000c66cbd800 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=61187, stack(0x0000000177528000,0x000000017792b000)] + 0x0000000c66cbde00 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=49667, stack(0x0000000177934000,0x0000000177d37000)] + 0x0000000c66cbe400 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=60675, stack(0x0000000177d40000,0x0000000178143000)] + 0x0000000c66cbea00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=60419, stack(0x000000017814c000,0x000000017854f000)] + 0x0000000c66ccd200 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50435, stack(0x0000000178558000,0x000000017895b000)] + 0x0000000c66ccf000 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=59651, stack(0x0000000178964000,0x0000000178d67000)] + 0x0000000c66ccf600 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=59395, stack(0x0000000178d70000,0x0000000179173000)] + 0x0000000c66d1c600 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=50947, stack(0x000000017917c000,0x000000017957f000)] + 0x0000000c66d1cc00 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=51459, stack(0x0000000179588000,0x000000017998b000)] + 0x0000000c66d1d800 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=58627, stack(0x0000000179994000,0x0000000179d97000)] + 0x0000000c66d1de00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=51971, stack(0x0000000179da0000,0x000000017a1a3000)] + 0x0000000c66d1e400 JavaThread "task-abort-timer" daemon [_thread_blocked, id=52227, stack(0x000000017a1ac000,0x000000017a5af000)] + 0x0000000c66d1f000 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=57859, stack(0x000000017a5b8000,0x000000017a9bb000)] + 0x0000000c66d1f600 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52483, stack(0x000000017a9c4000,0x000000017adc7000)] + 0x0000000c66d68c00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=52995, stack(0x000000017add0000,0x000000017b1d3000)] + 0x0000000c66d69200 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=53507, stack(0x000000017b1dc000,0x000000017b5df000)] + 0x0000000c66d69e00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=54019, stack(0x000000017b5e8000,0x000000017b9eb000)] + 0x0000000c66d6aa00 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54531, stack(0x000000017b9f4000,0x000000017bdf7000)] + 0x0000000c66d6b000 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=55043, stack(0x000000017be00000,0x000000017c203000)] + 0x0000000c66d6b600 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=55299, stack(0x000000017c20c000,0x000000017c60f000)] + 0x0000000c66dd8000 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55555, stack(0x000000017c618000,0x000000017ca1b000)] + 0x0000000c66de4600 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=55811, stack(0x000000017ca24000,0x000000017ce27000)] + 0x0000000c66de5e00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=56323, stack(0x000000017ce30000,0x000000017d233000)] + 0x0000000c66ec1800 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=65795, stack(0x000000017d648000,0x000000017da4b000)] + 0x0000000c66530c00 JavaThread "process reaper" daemon [_thread_blocked, id=34067, stack(0x0000000171578000,0x00000001715af000)] + 0x0000000c67a99800 JavaThread "Executor task launch worker for task 8.0 in stage 127.0 (TID 609)" daemon [_thread_blocked, id=43331, stack(0x0000000170b48000,0x0000000170f4b000)] + 0x0000000c67a99e00 JavaThread "Executor task launch worker for task 5.0 in stage 267.0 (TID 1087)" daemon [_thread_blocked, id=42787, stack(0x000000017da54000,0x000000017de57000)] +=>0x0000000c67a9a400 JavaThread "Executor task launch worker for task 0.0 in stage 268.0 (TID 1092)" daemon [_thread_in_vm, id=29447, stack(0x000000017de60000,0x000000017e263000)] + 0x0000000c67a9aa00 JavaThread "Executor task launch worker for task 7.0 in stage 267.0 (TID 1089)" daemon [_thread_blocked, id=86275, stack(0x000000017e26c000,0x000000017e66f000)] + 0x0000000c67a9b000 JavaThread "Executor task launch worker for task 9.0 in stage 267.0 (TID 1091)" daemon [_thread_blocked, id=66307, stack(0x000000017e678000,0x000000017ea7b000)] + 0x0000000c676b0600 JavaThread "process reaper" daemon [_thread_blocked, id=67075, stack(0x000000016ec44000,0x000000016ec7b000)] + 0x0000000c676b2400 JavaThread "process reaper" daemon [_thread_blocked, id=67587, stack(0x000000016ec84000,0x000000016ecbb000)] + 0x0000000c67aab600 JavaThread "process reaper" daemon [_thread_blocked, id=84995, stack(0x00000001715b8000,0x00000001715ef000)] + 0x0000000c67aaaa00 JavaThread "process reaper" daemon [_thread_blocked, id=84483, stack(0x00000001715f8000,0x000000017162f000)] + 0x0000000c67130600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=68403, stack(0x000000017ea84000,0x000000017ee87000)] + 0x0000000c67130c00 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=83983, stack(0x000000017ee90000,0x000000017f293000)] + 0x0000000c67131200 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=66599, stack(0x000000017f29c000,0x000000017f69f000)] + 0x0000000c67131800 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=85519, stack(0x000000017f6a8000,0x000000017faab000)] + 0x0000000c65d64600 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=66875, stack(0x000000017fab4000,0x000000017feb7000)] + 0x0000000c65d64c00 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=68635, stack(0x0000000300004000,0x0000000300407000)] + 0x0000000c65d65200 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=68867, stack(0x0000000300410000,0x0000000300813000)] + 0x0000000c65d65800 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=82691, stack(0x000000030081c000,0x0000000300c1f000)] + 0x0000000c65d65e00 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=82179, stack(0x0000000300c28000,0x000000030102b000)] + 0x0000000c65e5a400 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=69379, stack(0x0000000301034000,0x0000000301437000)] + 0x0000000c65e59e00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=81923, stack(0x0000000301440000,0x0000000301843000)] + 0x0000000c65e5aa00 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=81667, stack(0x000000030184c000,0x0000000301c4f000)] + 0x0000000c65e5b000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=70403, stack(0x0000000301c58000,0x000000030205b000)] + 0x0000000c65e5b600 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=81155, stack(0x0000000302064000,0x0000000302467000)] + 0x0000000c64688000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=65031, stack(0x00000003034d0000,0x00000003038d3000)] + 0x0000000c6862f000 JavaThread "block-manager-storage-async-thread-pool-6" daemon [_thread_blocked, id=77571, stack(0x000000030410c000,0x000000030450f000)] + 0x0000000c677e6a00 JavaThread "block-manager-storage-async-thread-pool-7" daemon [_thread_blocked, id=73731, stack(0x0000000304518000,0x000000030491b000)] + 0x0000000c677e4000 JavaThread "block-manager-storage-async-thread-pool-8" daemon [_thread_blocked, id=77059, stack(0x0000000304924000,0x0000000304d27000)] + 0x0000000c677e4c00 JavaThread "block-manager-ask-thread-pool-4" daemon [_thread_blocked, id=74499, stack(0x0000000304d30000,0x0000000305133000)] + 0x0000000c67132a00 JavaThread "block-manager-ask-thread-pool-5" daemon [_thread_blocked, id=76547, stack(0x000000030513c000,0x000000030553f000)] + 0x0000000c654ad200 JavaThread "QueryStageCreator-1" daemon [_thread_blocked, id=72203, stack(0x0000000170524000,0x0000000170927000)] + 0x0000000c654ad800 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=79387, stack(0x0000000170930000,0x0000000170b33000)] + 0x0000000c654ade00 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=72467, stack(0x0000000174274000,0x0000000174477000)] + 0x0000000c654aea00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=45843, stack(0x0000000302eac000,0x00000003030af000)] + 0x0000000c654af000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=27155, stack(0x00000003032c4000,0x00000003034c7000)] + 0x0000000c654af600 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=78867, stack(0x00000003038dc000,0x0000000303adf000)] + 0x0000000c688a7600 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=26899, stack(0x0000000303ae8000,0x0000000303ceb000)] + 0x0000000c677e5800 JavaThread "C2 CompilerThread4" daemon [_thread_blocked, id=72975, stack(0x0000000303cf4000,0x0000000303ef7000)] + 0x0000000c677e5e00 JavaThread "C2 CompilerThread5" daemon [_thread_blocked, id=73223, stack(0x0000000303f00000,0x0000000304103000)] + 0x0000000c677e7000 JavaThread "QueryStageCreator-2" daemon [_thread_blocked, id=75779, stack(0x0000000306190000,0x0000000306593000)] + 0x0000000c677e7600 JavaThread "Timer-1" [_thread_blocked, id=65555, stack(0x000000017d23c000,0x000000017d63f000)] + 0x0000000c67aaa400 JavaThread "QueryStageCreator-3" daemon [_thread_blocked, id=87315, stack(0x000000030659c000,0x000000030699f000)] + 0x0000000c6890b600 JavaThread "block-manager-storage-async-thread-pool-9" daemon [_thread_blocked, id=87571, stack(0x00000003069a8000,0x0000000306dab000)] + 0x0000000c68908000 JavaThread "block-manager-storage-async-thread-pool-10" daemon [_thread_blocked, id=88067, stack(0x0000000306db4000,0x00000003071b7000)] + 0x0000000c65e59200 JavaThread "block-manager-storage-async-thread-pool-11" daemon [_thread_blocked, id=88579, stack(0x00000003071c0000,0x00000003075c3000)] + 0x0000000c646f8c00 JavaThread "block-manager-storage-async-thread-pool-12" daemon [_thread_blocked, id=130563, stack(0x00000003075cc000,0x00000003079cf000)] + 0x0000000c646f8600 JavaThread "block-manager-ask-thread-pool-6" daemon [_thread_blocked, id=130307, stack(0x00000003079d8000,0x0000000307ddb000)] + 0x0000000c640f0000 JavaThread "block-manager-ask-thread-pool-7" daemon [_thread_blocked, id=89347, stack(0x0000000307de4000,0x00000003081e7000)] + 0x0000000c640f0600 JavaThread "block-manager-storage-async-thread-pool-13" daemon [_thread_blocked, id=129539, stack(0x00000003081f0000,0x00000003085f3000)] + 0x0000000c640f0c00 JavaThread "block-manager-storage-async-thread-pool-14" daemon [_thread_blocked, id=129283, stack(0x00000003085fc000,0x00000003089ff000)] + 0x0000000c640f1200 JavaThread "block-manager-storage-async-thread-pool-15" daemon [_thread_blocked, id=90115, stack(0x0000000308a08000,0x0000000308e0b000)] + 0x0000000c640f1800 JavaThread "block-manager-storage-async-thread-pool-16" daemon [_thread_blocked, id=129027, stack(0x0000000308e14000,0x0000000309217000)] + 0x0000000c640f1e00 JavaThread "block-manager-storage-async-thread-pool-17" daemon [_thread_blocked, id=128771, stack(0x0000000309220000,0x0000000309623000)] + 0x0000000c640f2400 JavaThread "block-manager-ask-thread-pool-8" daemon [_thread_blocked, id=128515, stack(0x000000030962c000,0x0000000309a2f000)] + 0x0000000c640f2a00 JavaThread "block-manager-ask-thread-pool-9" daemon [_thread_blocked, id=128003, stack(0x0000000309a38000,0x0000000309e3b000)] + 0x0000000c640f3000 JavaThread "block-manager-ask-thread-pool-10" daemon [_thread_blocked, id=91139, stack(0x0000000309e44000,0x000000030a247000)] + 0x0000000c640f3600 JavaThread "block-manager-ask-thread-pool-11" daemon [_thread_blocked, id=127491, stack(0x000000030a250000,0x000000030a653000)] + 0x0000000c667b4000 JavaThread "block-manager-storage-async-thread-pool-18" daemon [_thread_blocked, id=126979, stack(0x000000030a65c000,0x000000030aa5f000)] + 0x0000000c667b4600 JavaThread "block-manager-storage-async-thread-pool-19" daemon [_thread_blocked, id=91907, stack(0x000000030aa68000,0x000000030ae6b000)] + 0x0000000c667b4c00 JavaThread "block-manager-storage-async-thread-pool-20" daemon [_thread_blocked, id=92419, stack(0x000000030ae74000,0x000000030b277000)] + 0x0000000c667b5200 JavaThread "block-manager-ask-thread-pool-12" daemon [_thread_blocked, id=92931, stack(0x000000030b280000,0x000000030b683000)] + 0x0000000c667b5800 JavaThread "block-manager-ask-thread-pool-13" daemon [_thread_blocked, id=93443, stack(0x000000030b68c000,0x000000030ba8f000)] + 0x0000000c667b5e00 JavaThread "block-manager-storage-async-thread-pool-21" daemon [_thread_blocked, id=126723, stack(0x000000030ba98000,0x000000030be9b000)] + 0x0000000c667b6400 JavaThread "block-manager-storage-async-thread-pool-22" daemon [_thread_blocked, id=126467, stack(0x000000030bea4000,0x000000030c2a7000)] + 0x0000000c667b6a00 JavaThread "block-manager-storage-async-thread-pool-23" daemon [_thread_blocked, id=126211, stack(0x000000030c2b0000,0x000000030c6b3000)] + 0x0000000c667b7000 JavaThread "block-manager-ask-thread-pool-14" daemon [_thread_blocked, id=125699, stack(0x000000030c6bc000,0x000000030cabf000)] + 0x0000000c667b7600 JavaThread "block-manager-ask-thread-pool-15" daemon [_thread_blocked, id=125443, stack(0x000000030cac8000,0x000000030cecb000)] + 0x0000000c6557c000 JavaThread "block-manager-storage-async-thread-pool-24" daemon [_thread_blocked, id=125187, stack(0x000000030ced4000,0x000000030d2d7000)] + 0x0000000c6557c600 JavaThread "block-manager-storage-async-thread-pool-25" daemon [_thread_blocked, id=95235, stack(0x000000030d2e0000,0x000000030d6e3000)] + 0x0000000c6557cc00 JavaThread "block-manager-storage-async-thread-pool-26" daemon [_thread_blocked, id=95491, stack(0x000000030d6ec000,0x000000030daef000)] + 0x0000000c6557d200 JavaThread "block-manager-ask-thread-pool-16" daemon [_thread_blocked, id=96003, stack(0x000000030daf8000,0x000000030defb000)] + 0x0000000c6557d800 JavaThread "block-manager-ask-thread-pool-17" daemon [_thread_blocked, id=96515, stack(0x000000030df04000,0x000000030e307000)] + 0x0000000c6557de00 JavaThread "block-manager-storage-async-thread-pool-27" daemon [_thread_blocked, id=124419, stack(0x000000030e310000,0x000000030e713000)] + 0x0000000c6557e400 JavaThread "block-manager-storage-async-thread-pool-28" daemon [_thread_blocked, id=96771, stack(0x000000030e71c000,0x000000030eb1f000)] + 0x0000000c6557ea00 JavaThread "block-manager-storage-async-thread-pool-29" daemon [_thread_blocked, id=97283, stack(0x000000030eb28000,0x000000030ef2b000)] + 0x0000000c6557f000 JavaThread "block-manager-ask-thread-pool-18" daemon [_thread_blocked, id=123907, stack(0x000000030ef34000,0x000000030f337000)] + 0x0000000c6557f600 JavaThread "block-manager-ask-thread-pool-19" daemon [_thread_blocked, id=98051, stack(0x000000030f340000,0x000000030f743000)] + 0x0000000c640fc000 JavaThread "block-manager-storage-async-thread-pool-30" daemon [_thread_blocked, id=98307, stack(0x000000030f74c000,0x000000030fb4f000)] + 0x0000000c640fc600 JavaThread "block-manager-storage-async-thread-pool-31" daemon [_thread_blocked, id=98563, stack(0x000000030fb58000,0x000000030ff5b000)] + 0x0000000c640fcc00 JavaThread "block-manager-storage-async-thread-pool-32" daemon [_thread_blocked, id=99075, stack(0x000000030ff64000,0x0000000310367000)] + 0x0000000c640fd200 JavaThread "block-manager-ask-thread-pool-20" daemon [_thread_blocked, id=99587, stack(0x0000000310370000,0x0000000310773000)] + 0x0000000c640fd800 JavaThread "block-manager-ask-thread-pool-21" daemon [_thread_blocked, id=100099, stack(0x000000031077c000,0x0000000310b7f000)] + 0x0000000c640ff000 JavaThread "QueryStageCreator-4" daemon [_thread_blocked, id=122883, stack(0x0000000310b88000,0x0000000310f8b000)] + 0x0000000c655bd200 JavaThread "QueryStageCreator-5" daemon [_thread_blocked, id=122371, stack(0x0000000310f94000,0x0000000311397000)] + 0x0000000c655be400 JavaThread "QueryStageCreator-6" daemon [_thread_blocked, id=121875, stack(0x00000003113a0000,0x00000003117a3000)] + 0x0000000c655bea00 JavaThread "block-manager-storage-async-thread-pool-33" daemon [_thread_blocked, id=121619, stack(0x00000003117ac000,0x0000000311baf000)] + 0x0000000c655bf000 JavaThread "block-manager-storage-async-thread-pool-34" daemon [_thread_blocked, id=121091, stack(0x0000000311bb8000,0x0000000311fbb000)] + 0x0000000c655bf600 JavaThread "block-manager-storage-async-thread-pool-35" daemon [_thread_blocked, id=100611, stack(0x0000000311fc4000,0x00000003123c7000)] + 0x0000000c66de6a00 JavaThread "block-manager-storage-async-thread-pool-36" daemon [_thread_blocked, id=120323, stack(0x00000003123d0000,0x00000003127d3000)] + 0x0000000c655e4000 JavaThread "block-manager-ask-thread-pool-22" daemon [_thread_blocked, id=119811, stack(0x00000003127dc000,0x0000000312bdf000)] + 0x0000000c655e4600 JavaThread "block-manager-storage-async-thread-pool-37" daemon [_thread_blocked, id=100867, stack(0x0000000312be8000,0x0000000312feb000)] + 0x0000000c655e4c00 JavaThread "block-manager-ask-thread-pool-23" daemon [_thread_blocked, id=101123, stack(0x0000000312ff4000,0x00000003133f7000)] + 0x0000000c655e5200 JavaThread "block-manager-storage-async-thread-pool-38" daemon [_thread_blocked, id=101635, stack(0x0000000313400000,0x0000000313803000)] + 0x0000000c655e5800 JavaThread "block-manager-ask-thread-pool-24" daemon [_thread_blocked, id=119043, stack(0x000000031380c000,0x0000000313c0f000)] + 0x0000000c655e5e00 JavaThread "block-manager-ask-thread-pool-25" daemon [_thread_blocked, id=102403, stack(0x0000000313c18000,0x000000031401b000)] + 0x0000000c655e6400 JavaThread "block-manager-storage-async-thread-pool-39" daemon [_thread_blocked, id=102659, stack(0x0000000314024000,0x0000000314427000)] + 0x0000000c655e6a00 JavaThread "block-manager-storage-async-thread-pool-40" daemon [_thread_blocked, id=118531, stack(0x0000000314430000,0x0000000314833000)] + 0x0000000c655e7000 JavaThread "block-manager-storage-async-thread-pool-41" daemon [_thread_blocked, id=118019, stack(0x000000031483c000,0x0000000314c3f000)] + 0x0000000c655e7600 JavaThread "block-manager-ask-thread-pool-26" daemon [_thread_blocked, id=117763, stack(0x0000000314c48000,0x000000031504b000)] + 0x0000000c64724000 JavaThread "block-manager-ask-thread-pool-27" daemon [_thread_blocked, id=103427, stack(0x0000000315054000,0x0000000315457000)] + 0x0000000c64724600 JavaThread "block-manager-storage-async-thread-pool-42" daemon [_thread_blocked, id=103683, stack(0x0000000315460000,0x0000000315863000)] + 0x0000000c64724c00 JavaThread "block-manager-storage-async-thread-pool-43" daemon [_thread_blocked, id=103939, stack(0x000000031586c000,0x0000000315c6f000)] + 0x0000000c64725200 JavaThread "block-manager-storage-async-thread-pool-44" daemon [_thread_blocked, id=116739, stack(0x0000000315c78000,0x000000031607b000)] + 0x0000000c64725800 JavaThread "block-manager-storage-async-thread-pool-45" daemon [_thread_blocked, id=104707, stack(0x0000000316084000,0x0000000316487000)] + 0x0000000c64725e00 JavaThread "block-manager-ask-thread-pool-28" daemon [_thread_blocked, id=104963, stack(0x0000000316490000,0x0000000316893000)] + 0x0000000c64726400 JavaThread "block-manager-storage-async-thread-pool-46" daemon [_thread_blocked, id=105219, stack(0x000000031689c000,0x0000000316c9f000)] + 0x0000000c64726a00 JavaThread "block-manager-ask-thread-pool-29" daemon [_thread_blocked, id=105731, stack(0x0000000316ca8000,0x00000003170ab000)] + 0x0000000c64727000 JavaThread "block-manager-storage-async-thread-pool-47" daemon [_thread_blocked, id=106243, stack(0x00000003170b4000,0x00000003174b7000)] + 0x0000000c64727600 JavaThread "block-manager-ask-thread-pool-30" daemon [_thread_blocked, id=106755, stack(0x00000003174c0000,0x00000003178c3000)] + 0x0000000c6412c000 JavaThread "block-manager-ask-thread-pool-31" daemon [_thread_blocked, id=107011, stack(0x00000003178cc000,0x0000000317ccf000)] + 0x0000000c6412c600 JavaThread "block-manager-storage-async-thread-pool-48" daemon [_thread_blocked, id=107267, stack(0x0000000317cd8000,0x00000003180db000)] + 0x0000000c6412cc00 JavaThread "block-manager-storage-async-thread-pool-49" daemon [_thread_blocked, id=115203, stack(0x00000003180e4000,0x00000003184e7000)] + 0x0000000c6412d200 JavaThread "block-manager-storage-async-thread-pool-50" daemon [_thread_blocked, id=107779, stack(0x00000003184f0000,0x00000003188f3000)] + 0x0000000c6412d800 JavaThread "block-manager-ask-thread-pool-32" daemon [_thread_blocked, id=114947, stack(0x00000003188fc000,0x0000000318cff000)] + 0x0000000c6412e400 JavaThread "block-manager-ask-thread-pool-33" daemon [_thread_blocked, id=114691, stack(0x0000000318d08000,0x000000031910b000)] + 0x0000000c6412de00 JavaThread "block-manager-storage-async-thread-pool-51" daemon [_thread_blocked, id=108547, stack(0x0000000319114000,0x0000000319517000)] + 0x0000000c6412ea00 JavaThread "block-manager-storage-async-thread-pool-52" daemon [_thread_blocked, id=114179, stack(0x0000000319520000,0x0000000319923000)] + 0x0000000c6412f000 JavaThread "block-manager-storage-async-thread-pool-53" daemon [_thread_blocked, id=113667, stack(0x000000031992c000,0x0000000319d2f000)] + 0x0000000c64134600 JavaThread "block-manager-ask-thread-pool-34" daemon [_thread_blocked, id=113155, stack(0x0000000319d38000,0x000000031a13b000)] + 0x0000000c64134000 JavaThread "block-manager-ask-thread-pool-35" daemon [_thread_blocked, id=109315, stack(0x000000031a144000,0x000000031a547000)] + 0x0000000c64134c00 JavaThread "block-manager-storage-async-thread-pool-54" daemon [_thread_blocked, id=109571, stack(0x000000031a550000,0x000000031a953000)] + 0x0000000c64135200 JavaThread "block-manager-storage-async-thread-pool-55" daemon [_thread_blocked, id=112387, stack(0x000000031a95c000,0x000000031ad5f000)] + 0x0000000c64135800 JavaThread "block-manager-storage-async-thread-pool-56" daemon [_thread_blocked, id=109827, stack(0x000000031ad68000,0x000000031b16b000)] + 0x0000000c64135e00 JavaThread "block-manager-ask-thread-pool-36" daemon [_thread_blocked, id=111619, stack(0x000000031b174000,0x000000031b577000)] + 0x0000000c64136400 JavaThread "block-manager-ask-thread-pool-37" daemon [_thread_blocked, id=111363, stack(0x000000031b580000,0x000000031b983000)] + 0x0000000c64136a00 JavaThread "block-manager-storage-async-thread-pool-57" daemon [_thread_blocked, id=110851, stack(0x000000031b98c000,0x000000031bd8f000)] + 0x0000000c64137000 JavaThread "block-manager-storage-async-thread-pool-58" daemon [_thread_blocked, id=110339, stack(0x000000031bd98000,0x000000031c19b000)] + 0x0000000c64137600 JavaThread "block-manager-storage-async-thread-pool-59" daemon [_thread_blocked, id=131331, stack(0x000000031c1a4000,0x000000031c5a7000)] + 0x0000000c655ec600 JavaThread "block-manager-ask-thread-pool-38" daemon [_thread_blocked, id=174083, stack(0x000000031c5b0000,0x000000031c9b3000)] + 0x0000000c655ec000 JavaThread "block-manager-ask-thread-pool-39" daemon [_thread_blocked, id=131843, stack(0x000000031c9bc000,0x000000031cdbf000)] + 0x0000000c655ecc00 JavaThread "block-manager-storage-async-thread-pool-60" daemon [_thread_blocked, id=132355, stack(0x000000031cdc8000,0x000000031d1cb000)] + 0x0000000c655ed200 JavaThread "block-manager-storage-async-thread-pool-61" daemon [_thread_blocked, id=132611, stack(0x000000031d1d4000,0x000000031d5d7000)] + 0x0000000c655ed800 JavaThread "block-manager-storage-async-thread-pool-62" daemon [_thread_blocked, id=132867, stack(0x000000031d5e0000,0x000000031d9e3000)] + 0x0000000c655ede00 JavaThread "block-manager-storage-async-thread-pool-63" daemon [_thread_blocked, id=173315, stack(0x000000031d9ec000,0x000000031ddef000)] + 0x0000000c655ee400 JavaThread "block-manager-ask-thread-pool-40" daemon [_thread_blocked, id=133635, stack(0x000000031ddf8000,0x000000031e1fb000)] + 0x0000000c655eea00 JavaThread "block-manager-storage-async-thread-pool-64" daemon [_thread_blocked, id=134147, stack(0x000000031e204000,0x000000031e607000)] + 0x0000000c655ef000 JavaThread "block-manager-ask-thread-pool-41" daemon [_thread_blocked, id=172803, stack(0x000000031e610000,0x000000031ea13000)] + 0x0000000c655ef600 JavaThread "block-manager-storage-async-thread-pool-65" daemon [_thread_blocked, id=172291, stack(0x000000031ea1c000,0x000000031ee1f000)] + 0x0000000c655f0600 JavaThread "block-manager-ask-thread-pool-42" daemon [_thread_blocked, id=171779, stack(0x000000031ee28000,0x000000031f22b000)] + 0x0000000c655f0000 JavaThread "block-manager-ask-thread-pool-43" daemon [_thread_blocked, id=134403, stack(0x000000031f234000,0x000000031f637000)] + 0x0000000c655f1e00 JavaThread "QueryStageCreator-7" daemon [_thread_blocked, id=171011, stack(0x000000031f640000,0x000000031fa43000)] + 0x0000000c655f3600 JavaThread "QueryStageCreator-8" daemon [_thread_blocked, id=134915, stack(0x000000031fa4c000,0x000000031fe4f000)] + 0x0000000c64738600 JavaThread "QueryStageCreator-9" daemon [_thread_blocked, id=170531, stack(0x000000031fe58000,0x000000032025b000)] + 0x0000000c6473b600 JavaThread "block-manager-storage-async-thread-pool-66" daemon [_thread_blocked, id=135179, stack(0x0000000320264000,0x0000000320667000)] + 0x0000000c6473b000 JavaThread "block-manager-storage-async-thread-pool-67" daemon [_thread_blocked, id=169731, stack(0x0000000320670000,0x0000000320a73000)] + 0x0000000c655f2400 JavaThread "block-manager-storage-async-thread-pool-68" daemon [_thread_blocked, id=169219, stack(0x0000000320a7c000,0x0000000320e7f000)] + 0x0000000c6412f600 JavaThread "block-manager-ask-thread-pool-44" daemon [_thread_blocked, id=168707, stack(0x0000000320e88000,0x000000032128b000)] + 0x0000000c64754000 JavaThread "block-manager-ask-thread-pool-45" daemon [_thread_blocked, id=168451, stack(0x0000000321294000,0x0000000321697000)] + 0x0000000c64754600 JavaThread "block-manager-storage-async-thread-pool-69" daemon [_thread_blocked, id=167939, stack(0x00000003216a0000,0x0000000321aa3000)] + 0x0000000c64754c00 JavaThread "block-manager-storage-async-thread-pool-70" daemon [_thread_blocked, id=135939, stack(0x0000000321aac000,0x0000000321eaf000)] + 0x0000000c64755200 JavaThread "block-manager-storage-async-thread-pool-71" daemon [_thread_blocked, id=167683, stack(0x0000000321eb8000,0x00000003222bb000)] + 0x0000000c64755800 JavaThread "block-manager-ask-thread-pool-46" daemon [_thread_blocked, id=136451, stack(0x00000003222c4000,0x00000003226c7000)] + 0x0000000c64755e00 JavaThread "block-manager-ask-thread-pool-47" daemon [_thread_blocked, id=136707, stack(0x00000003226d0000,0x0000000322ad3000)] + 0x0000000c64756400 JavaThread "block-manager-storage-async-thread-pool-72" daemon [_thread_blocked, id=137219, stack(0x0000000322adc000,0x0000000322edf000)] + 0x0000000c64756a00 JavaThread "block-manager-storage-async-thread-pool-73" daemon [_thread_blocked, id=137475, stack(0x0000000322ee8000,0x00000003232eb000)] + 0x0000000c64757000 JavaThread "block-manager-storage-async-thread-pool-74" daemon [_thread_blocked, id=166403, stack(0x00000003232f4000,0x00000003236f7000)] + 0x0000000c64757600 JavaThread "block-manager-storage-async-thread-pool-75" daemon [_thread_blocked, id=166147, stack(0x0000000323700000,0x0000000323b03000)] + 0x0000000c64758000 JavaThread "block-manager-ask-thread-pool-48" daemon [_thread_blocked, id=165635, stack(0x0000000323b0c000,0x0000000323f0f000)] + 0x0000000c64758600 JavaThread "block-manager-ask-thread-pool-49" daemon [_thread_blocked, id=137987, stack(0x0000000323f18000,0x000000032431b000)] + 0x0000000c64758c00 JavaThread "block-manager-storage-async-thread-pool-76" daemon [_thread_blocked, id=138243, stack(0x0000000324324000,0x0000000324727000)] + 0x0000000c64759200 JavaThread "block-manager-storage-async-thread-pool-77" daemon [_thread_blocked, id=138755, stack(0x0000000324730000,0x0000000324b33000)] + 0x0000000c64759800 JavaThread "block-manager-ask-thread-pool-50" daemon [_thread_blocked, id=139267, stack(0x0000000324b3c000,0x0000000324f3f000)] + 0x0000000c64759e00 JavaThread "block-manager-ask-thread-pool-51" daemon [_thread_blocked, id=164867, stack(0x0000000324f48000,0x000000032534b000)] + 0x0000000c6475a400 JavaThread "block-manager-storage-async-thread-pool-78" daemon [_thread_blocked, id=164355, stack(0x0000000325354000,0x0000000325757000)] + 0x0000000c6475aa00 JavaThread "block-manager-storage-async-thread-pool-79" daemon [_thread_blocked, id=163843, stack(0x0000000325760000,0x0000000325b63000)] + 0x0000000c6475b000 JavaThread "block-manager-storage-async-thread-pool-80" daemon [_thread_blocked, id=140035, stack(0x0000000325b6c000,0x0000000325f6f000)] + 0x0000000c6475b600 JavaThread "block-manager-ask-thread-pool-52" daemon [_thread_blocked, id=140547, stack(0x0000000325f78000,0x000000032637b000)] + 0x0000000c65600000 JavaThread "block-manager-ask-thread-pool-53" daemon [_thread_blocked, id=140803, stack(0x0000000326384000,0x0000000326787000)] + 0x0000000c65600600 JavaThread "block-manager-storage-async-thread-pool-81" daemon [_thread_blocked, id=141315, stack(0x0000000326790000,0x0000000326b93000)] + 0x0000000c65601200 JavaThread "block-manager-storage-async-thread-pool-82" daemon [_thread_blocked, id=141571, stack(0x0000000326b9c000,0x0000000326f9f000)] + 0x0000000c65601800 JavaThread "block-manager-storage-async-thread-pool-83" daemon [_thread_blocked, id=163075, stack(0x0000000326fa8000,0x00000003273ab000)] + 0x0000000c65601e00 JavaThread "block-manager-ask-thread-pool-54" daemon [_thread_blocked, id=142339, stack(0x00000003273b4000,0x00000003277b7000)] + 0x0000000c65602400 JavaThread "block-manager-ask-thread-pool-55" daemon [_thread_blocked, id=162563, stack(0x00000003277c0000,0x0000000327bc3000)] + 0x0000000c65602a00 JavaThread "block-manager-storage-async-thread-pool-84" daemon [_thread_blocked, id=162051, stack(0x0000000327bcc000,0x0000000327fcf000)] + 0x0000000c65603000 JavaThread "block-manager-storage-async-thread-pool-85" daemon [_thread_blocked, id=142851, stack(0x0000000327fd8000,0x00000003283db000)] + 0x0000000c65603600 JavaThread "block-manager-storage-async-thread-pool-86" daemon [_thread_blocked, id=143363, stack(0x00000003283e4000,0x00000003287e7000)] + 0x0000000c65608000 JavaThread "block-manager-storage-async-thread-pool-87" daemon [_thread_blocked, id=161539, stack(0x00000003287f0000,0x0000000328bf3000)] + 0x0000000c65608600 JavaThread "block-manager-ask-thread-pool-56" daemon [_thread_blocked, id=161283, stack(0x0000000328bfc000,0x0000000328fff000)] + 0x0000000c65608c00 JavaThread "block-manager-storage-async-thread-pool-88" daemon [_thread_blocked, id=161027, stack(0x0000000329008000,0x000000032940b000)] + 0x0000000c65609200 JavaThread "block-manager-storage-async-thread-pool-89" daemon [_thread_blocked, id=160771, stack(0x0000000329414000,0x0000000329817000)] + 0x0000000c65609800 JavaThread "block-manager-ask-thread-pool-57" daemon [_thread_blocked, id=160515, stack(0x0000000329820000,0x0000000329c23000)] + 0x0000000c65609e00 JavaThread "block-manager-storage-async-thread-pool-90" daemon [_thread_blocked, id=144899, stack(0x0000000329c2c000,0x000000032a02f000)] + 0x0000000c6560a400 JavaThread "block-manager-ask-thread-pool-58" daemon [_thread_blocked, id=145155, stack(0x000000032a038000,0x000000032a43b000)] + 0x0000000c6560aa00 JavaThread "block-manager-ask-thread-pool-59" daemon [_thread_blocked, id=160003, stack(0x000000032a444000,0x000000032a847000)] + 0x0000000c6560b000 JavaThread "block-manager-storage-async-thread-pool-91" daemon [_thread_blocked, id=159747, stack(0x000000032a850000,0x000000032ac53000)] + 0x0000000c6560b600 JavaThread "block-manager-storage-async-thread-pool-92" daemon [_thread_blocked, id=159235, stack(0x000000032ac5c000,0x000000032b05f000)] + 0x0000000c64170000 JavaThread "block-manager-ask-thread-pool-60" daemon [_thread_blocked, id=158979, stack(0x000000032b068000,0x000000032b46b000)] + 0x0000000c64170600 JavaThread "block-manager-ask-thread-pool-61" daemon [_thread_blocked, id=146435, stack(0x000000032b474000,0x000000032b877000)] + 0x0000000c64170c00 JavaThread "block-manager-storage-async-thread-pool-93" daemon [_thread_blocked, id=158467, stack(0x000000032b880000,0x000000032bc83000)] + 0x0000000c64171200 JavaThread "block-manager-storage-async-thread-pool-94" daemon [_thread_blocked, id=146691, stack(0x000000032bc8c000,0x000000032c08f000)] + 0x0000000c64171800 JavaThread "block-manager-storage-async-thread-pool-95" daemon [_thread_blocked, id=157955, stack(0x000000032c098000,0x000000032c49b000)] + 0x0000000c64171e00 JavaThread "block-manager-ask-thread-pool-62" daemon [_thread_blocked, id=147459, stack(0x000000032c4a4000,0x000000032c8a7000)] + 0x0000000c64172400 JavaThread "block-manager-ask-thread-pool-63" daemon [_thread_blocked, id=157699, stack(0x000000032c8b0000,0x000000032ccb3000)] + 0x0000000c64172a00 JavaThread "block-manager-storage-async-thread-pool-96" daemon [_thread_blocked, id=148227, stack(0x000000032ccbc000,0x000000032d0bf000)] + 0x0000000c64173000 JavaThread "block-manager-storage-async-thread-pool-97" daemon [_thread_blocked, id=157187, stack(0x000000032d0c8000,0x000000032d4cb000)] + 0x0000000c64173600 JavaThread "block-manager-storage-async-thread-pool-98" daemon [_thread_blocked, id=156675, stack(0x000000032d4d4000,0x000000032d8d7000)] + 0x0000000c64768000 JavaThread "block-manager-ask-thread-pool-64" daemon [_thread_blocked, id=148483, stack(0x000000032d8e0000,0x000000032dce3000)] + 0x0000000c64768600 JavaThread "block-manager-ask-thread-pool-65" daemon [_thread_blocked, id=155907, stack(0x000000032dcec000,0x000000032e0ef000)] + 0x0000000c64769200 JavaThread "QueryStageCreator-10" daemon [_thread_blocked, id=155395, stack(0x000000032e0f8000,0x000000032e4fb000)] + 0x0000000c64769800 JavaThread "QueryStageCreator-11" daemon [_thread_blocked, id=148739, stack(0x000000032e504000,0x000000032e907000)] + 0x0000000c641a0600 JavaThread "QueryStageCreator-12" daemon [_thread_blocked, id=154891, stack(0x000000032e910000,0x000000032ed13000)] + 0x0000000c641a1800 JavaThread "block-manager-storage-async-thread-pool-99" daemon [_thread_blocked, id=154659, stack(0x000000032ed1c000,0x000000032f11f000)] + 0x0000000c641a1e00 JavaThread "block-manager-ask-thread-pool-66" daemon [_thread_blocked, id=149251, stack(0x000000032f128000,0x000000032f52b000)] + 0x0000000c641a2400 JavaThread "block-manager-ask-thread-pool-67" daemon [_thread_blocked, id=153859, stack(0x000000032f534000,0x000000032f937000)] + 0x0000000c641a2a00 JavaThread "block-manager-ask-thread-pool-68" daemon [_thread_blocked, id=153347, stack(0x000000032f940000,0x000000032fd43000)] + 0x0000000c641a3000 JavaThread "block-manager-ask-thread-pool-69" daemon [_thread_blocked, id=152835, stack(0x000000032fd4c000,0x000000033014f000)] + 0x0000000c641a3600 JavaThread "block-manager-ask-thread-pool-70" daemon [_thread_blocked, id=152323, stack(0x0000000330158000,0x000000033055b000)] + 0x0000000c646fb600 JavaThread "block-manager-ask-thread-pool-71" daemon [_thread_blocked, id=151811, stack(0x0000000330564000,0x0000000330967000)] + 0x0000000c65dbde00 JavaThread "block-manager-ask-thread-pool-72" daemon [_thread_blocked, id=151555, stack(0x0000000330970000,0x0000000330d73000)] + 0x0000000c646e1200 JavaThread "block-manager-ask-thread-pool-73" daemon [_thread_blocked, id=150019, stack(0x0000000330d7c000,0x000000033117f000)] + 0x0000000c66531800 JavaThread "block-manager-ask-thread-pool-74" daemon [_thread_blocked, id=150787, stack(0x0000000331188000,0x000000033158b000)] + 0x0000000c66531200 JavaThread "block-manager-ask-thread-pool-75" daemon [_thread_blocked, id=150275, stack(0x0000000331594000,0x0000000331997000)] + 0x0000000c641a8000 JavaThread "block-manager-ask-thread-pool-76" daemon [_thread_blocked, id=174851, stack(0x00000003319a0000,0x0000000331da3000)] + 0x0000000c641a8600 JavaThread "block-manager-ask-thread-pool-77" daemon [_thread_blocked, id=261891, stack(0x0000000331dac000,0x00000003321af000)] + 0x0000000c641a8c00 JavaThread "block-manager-ask-thread-pool-78" daemon [_thread_blocked, id=261635, stack(0x00000003321b8000,0x00000003325bb000)] + 0x0000000c641a9200 JavaThread "block-manager-ask-thread-pool-79" daemon [_thread_blocked, id=261123, stack(0x00000003325c4000,0x00000003329c7000)] + 0x0000000c641a9800 JavaThread "block-manager-ask-thread-pool-80" daemon [_thread_blocked, id=260867, stack(0x00000003329d0000,0x0000000332dd3000)] + 0x0000000c641a9e00 JavaThread "block-manager-ask-thread-pool-81" daemon [_thread_blocked, id=176131, stack(0x0000000332ddc000,0x00000003331df000)] + 0x0000000c641aa400 JavaThread "block-manager-ask-thread-pool-82" daemon [_thread_blocked, id=260355, stack(0x00000003331e8000,0x00000003335eb000)] + 0x0000000c641aaa00 JavaThread "block-manager-ask-thread-pool-83" daemon [_thread_blocked, id=176387, stack(0x00000003335f4000,0x00000003339f7000)] + 0x0000000c641ab000 JavaThread "block-manager-ask-thread-pool-84" daemon [_thread_blocked, id=259587, stack(0x0000000333a00000,0x0000000333e03000)] + 0x0000000c641ab600 JavaThread "block-manager-ask-thread-pool-85" daemon [_thread_blocked, id=176643, stack(0x0000000333e0c000,0x000000033420f000)] + 0x0000000c6477c600 JavaThread "block-manager-ask-thread-pool-86" daemon [_thread_blocked, id=177155, stack(0x0000000334218000,0x000000033461b000)] + 0x0000000c6477d200 JavaThread "block-manager-ask-thread-pool-87" daemon [_thread_blocked, id=259075, stack(0x0000000334624000,0x0000000334a27000)] + 0x0000000c6477d800 JavaThread "QueryStageCreator-13" daemon [_thread_blocked, id=177923, stack(0x0000000334a30000,0x0000000334e33000)] + 0x0000000c6477e400 JavaThread "QueryStageCreator-14" daemon [_thread_blocked, id=258819, stack(0x0000000334e3c000,0x000000033523f000)] + 0x0000000c6477ea00 JavaThread "QueryStageCreator-15" daemon [_thread_blocked, id=258571, stack(0x0000000335248000,0x000000033564b000)] + 0x0000000c6477f600 JavaThread "block-manager-ask-thread-pool-88" daemon [_thread_blocked, id=179219, stack(0x0000000335654000,0x0000000335a57000)] + 0x0000000c64200000 JavaThread "block-manager-ask-thread-pool-89" daemon [_thread_blocked, id=179459, stack(0x0000000335a60000,0x0000000335e63000)] + 0x0000000c64200600 JavaThread "block-manager-ask-thread-pool-90" daemon [_thread_blocked, id=179715, stack(0x0000000335e6c000,0x000000033626f000)] + 0x0000000c64200c00 JavaThread "block-manager-ask-thread-pool-91" daemon [_thread_blocked, id=179971, stack(0x0000000336278000,0x000000033667b000)] + 0x0000000c64201200 JavaThread "block-manager-ask-thread-pool-92" daemon [_thread_blocked, id=180227, stack(0x0000000336684000,0x0000000336a87000)] + 0x0000000c64201800 JavaThread "block-manager-ask-thread-pool-93" daemon [_thread_blocked, id=257539, stack(0x0000000336a90000,0x0000000336e93000)] + 0x0000000c64201e00 JavaThread "block-manager-ask-thread-pool-94" daemon [_thread_blocked, id=180739, stack(0x0000000336e9c000,0x000000033729f000)] + 0x0000000c64202400 JavaThread "block-manager-ask-thread-pool-95" daemon [_thread_blocked, id=256771, stack(0x00000003372a8000,0x00000003376ab000)] + 0x0000000c64202a00 JavaThread "block-manager-ask-thread-pool-96" daemon [_thread_blocked, id=180995, stack(0x00000003376b4000,0x0000000337ab7000)] + 0x0000000c64203000 JavaThread "block-manager-ask-thread-pool-97" daemon [_thread_blocked, id=256003, stack(0x0000000337ac0000,0x0000000337ec3000)] + 0x0000000c64203600 JavaThread "block-manager-ask-thread-pool-98" daemon [_thread_blocked, id=255747, stack(0x0000000337ecc000,0x00000003382cf000)] + 0x0000000c64204000 JavaThread "block-manager-ask-thread-pool-99" daemon [_thread_blocked, id=181763, stack(0x00000003382d8000,0x00000003386db000)] + 0x0000000c64289e00 JavaThread "Executor task launch worker for task 8.0 in stage 267.0 (TID 1090)" daemon [_thread_blocked, id=255043, stack(0x00000003386e4000,0x0000000338ae7000)] + +Other Threads: + 0x0000000103401c20 VMThread "VM Thread" [stack: 0x000000016e194000,0x000000016e397000] [id=17923] + 0x0000000c68bf8600 WatcherThread [stack: 0x000000017136c000,0x000000017156f000] [id=42243] + 0x00000001033fb300 GCTaskThread "GC Thread#0" [stack: 0x000000016d758000,0x000000016d95b000] [id=14339] + 0x0000000c67438f00 GCTaskThread "GC Thread#1" [stack: 0x0000000171784000,0x0000000171987000] [id=34307] + 0x0000000c67439200 GCTaskThread "GC Thread#2" [stack: 0x0000000171990000,0x0000000171b93000] [id=34563] + 0x0000000c67439500 GCTaskThread "GC Thread#3" [stack: 0x0000000171b9c000,0x0000000171d9f000] [id=41475] + 0x0000000c67439800 GCTaskThread "GC Thread#4" [stack: 0x0000000171da8000,0x0000000171fab000] [id=35331] + 0x0000000c67439b00 GCTaskThread "GC Thread#5" [stack: 0x0000000171fb4000,0x00000001721b7000] [id=40963] + 0x0000000c67439e00 GCTaskThread "GC Thread#6" [stack: 0x00000001721c0000,0x00000001723c3000] [id=40451] + 0x0000000c6743a100 GCTaskThread "GC Thread#7" [stack: 0x00000001723cc000,0x00000001725cf000] [id=39939] + 0x0000000c6743a400 GCTaskThread "GC Thread#8" [stack: 0x00000001725d8000,0x00000001727db000] [id=39683] + 0x0000000c6743a700 GCTaskThread "GC Thread#9" [stack: 0x00000001727e4000,0x00000001729e7000] [id=39171] + 0x0000000c6743aa00 GCTaskThread "GC Thread#10" [stack: 0x00000001729f0000,0x0000000172bf3000] [id=36099] + 0x0000000c6743ad00 GCTaskThread "GC Thread#11" [stack: 0x0000000172bfc000,0x0000000172dff000] [id=36355] + 0x0000000c6743b000 GCTaskThread "GC Thread#12" [stack: 0x0000000172e08000,0x000000017300b000] [id=36611] + 0x0000000c6743b300 GCTaskThread "GC Thread#13" [stack: 0x0000000173014000,0x0000000173217000] [id=38147] + 0x0000000c6743b600 GCTaskThread "GC Thread#14" [stack: 0x0000000173220000,0x0000000173423000] [id=37635] + 0x0000000c6743b900 GCTaskThread "GC Thread#15" [stack: 0x000000017342c000,0x000000017362f000] [id=36867] + 0x0000000c6743bc00 GCTaskThread "GC Thread#16" [stack: 0x0000000173638000,0x000000017383b000] [id=43523] + 0x0000000c67450000 GCTaskThread "GC Thread#17" [stack: 0x0000000173844000,0x0000000173a47000] [id=44035] + 0x0000000c67450300 GCTaskThread "GC Thread#18" [stack: 0x0000000173a50000,0x0000000173c53000] [id=44547] + 0x0000000c67450600 GCTaskThread "GC Thread#19" [stack: 0x0000000173c5c000,0x0000000173e5f000] [id=45059] + 0x00000001033fbf00 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016d964000,0x000000016db67000] [id=13827] + 0x00000001033fc670 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016db70000,0x000000016dd73000] [id=13315] + 0x0000000c6755cf00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174480000,0x0000000174683000] [id=46083] + 0x0000000c6755d500 ConcurrentGCThread "G1 Conc#2" [stack: 0x000000017468c000,0x000000017488f000] [id=46339] + 0x0000000c6755d800 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000174898000,0x0000000174a9b000] [id=64259] + 0x0000000c6755db00 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000174aa4000,0x0000000174ca7000] [id=64003] + 0x00000001033fe810 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016dd7c000,0x000000016df7f000] [id=21507] + 0x00000001033fef10 ConcurrentGCThread "G1 Service" [stack: 0x000000016df88000,0x000000016e18b000] [id=16899] + +Threads with active compile tasks: + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x0000000700000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x000000c800000000-0x000000c800bc0000-0x000000c800bc0000), size 12320768, SharedBaseAddress: 0x000000c800000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x000000c801000000-0x000000c841000000, reserved size: 1073741824 +Narrow klass base: 0x000000c800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 954368K, used 430520K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 60 young (122880K), 4 survivors (8192K) + Metaspace used 170664K, committed 172096K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000700000000, 0x0000000700200000, 0x0000000700200000|100%|HS| |TAMS 0x0000000700200000, 0x0000000700200000| Complete +| 1|0x0000000700200000, 0x0000000700400000, 0x0000000700400000|100%|HS| |TAMS 0x0000000700400000, 0x0000000700400000| Complete +| 2|0x0000000700400000, 0x0000000700600000, 0x0000000700600000|100%|HC| |TAMS 0x0000000700600000, 0x0000000700600000| Complete +| 3|0x0000000700600000, 0x0000000700800000, 0x0000000700800000|100%|HS| |TAMS 0x0000000700800000, 0x0000000700800000| Complete +| 4|0x0000000700800000, 0x0000000700a00000, 0x0000000700a00000|100%|HC| |TAMS 0x0000000700a00000, 0x0000000700a00000| Complete +| 5|0x0000000700a00000, 0x0000000700c00000, 0x0000000700c00000|100%|HC| |TAMS 0x0000000700c00000, 0x0000000700c00000| Complete +| 6|0x0000000700c00000, 0x0000000700e00000, 0x0000000700e00000|100%|HC| |TAMS 0x0000000700e00000, 0x0000000700e00000| Complete +| 7|0x0000000700e00000, 0x0000000701000000, 0x0000000701000000|100%|HC| |TAMS 0x0000000701000000, 0x0000000701000000| Complete +| 8|0x0000000701000000, 0x0000000701200000, 0x0000000701200000|100%|HC| |TAMS 0x0000000701200000, 0x0000000701200000| Complete +| 9|0x0000000701200000, 0x0000000701400000, 0x0000000701400000|100%|HC| |TAMS 0x0000000701400000, 0x0000000701400000| Complete +| 10|0x0000000701400000, 0x0000000701600000, 0x0000000701600000|100%|HC| |TAMS 0x0000000701600000, 0x0000000701600000| Complete +| 11|0x0000000701600000, 0x0000000701800000, 0x0000000701800000|100%|HS| |TAMS 0x0000000701800000, 0x0000000701800000| Complete +| 12|0x0000000701800000, 0x0000000701a00000, 0x0000000701a00000|100%|HS| |TAMS 0x0000000701a00000, 0x0000000701a00000| Complete +| 13|0x0000000701a00000, 0x0000000701c00000, 0x0000000701c00000|100%|HC| |TAMS 0x0000000701c00000, 0x0000000701c00000| Complete +| 14|0x0000000701c00000, 0x0000000701e00000, 0x0000000701e00000|100%|HS| |TAMS 0x0000000701e00000, 0x0000000701e00000| Complete +| 15|0x0000000701e00000, 0x0000000702000000, 0x0000000702000000|100%|HC| |TAMS 0x0000000702000000, 0x0000000702000000| Complete +| 16|0x0000000702000000, 0x0000000702200000, 0x0000000702200000|100%|HS| |TAMS 0x0000000702200000, 0x0000000702200000| Complete +| 17|0x0000000702200000, 0x0000000702400000, 0x0000000702400000|100%|HS| |TAMS 0x0000000702400000, 0x0000000702400000| Complete +| 18|0x0000000702400000, 0x0000000702600000, 0x0000000702600000|100%|HC| |TAMS 0x0000000702600000, 0x0000000702600000| Complete +| 19|0x0000000702600000, 0x0000000702800000, 0x0000000702800000|100%| O| |TAMS 0x0000000702800000, 0x0000000702800000| Untracked +| 20|0x0000000702800000, 0x0000000702a00000, 0x0000000702a00000|100%| O| |TAMS 0x0000000702a00000, 0x0000000702a00000| Untracked +| 21|0x0000000702a00000, 0x0000000702c00000, 0x0000000702c00000|100%| O| |TAMS 0x0000000702c00000, 0x0000000702c00000| Untracked +| 22|0x0000000702c00000, 0x0000000702e00000, 0x0000000702e00000|100%| O| |TAMS 0x0000000702e00000, 0x0000000702e00000| Untracked +| 23|0x0000000702e00000, 0x0000000703000000, 0x0000000703000000|100%| O| |TAMS 0x0000000703000000, 0x0000000703000000| Untracked +| 24|0x0000000703000000, 0x0000000703200000, 0x0000000703200000|100%| O| |TAMS 0x0000000703200000, 0x0000000703200000| Untracked +| 25|0x0000000703200000, 0x0000000703400000, 0x0000000703400000|100%| O| |TAMS 0x0000000703400000, 0x0000000703400000| Untracked +| 26|0x0000000703400000, 0x0000000703600000, 0x0000000703600000|100%| O| |TAMS 0x0000000703600000, 0x0000000703600000| Untracked +| 27|0x0000000703600000, 0x0000000703800000, 0x0000000703800000|100%| O| |TAMS 0x0000000703800000, 0x0000000703800000| Untracked +| 28|0x0000000703800000, 0x0000000703a00000, 0x0000000703a00000|100%| O| |TAMS 0x0000000703a00000, 0x0000000703a00000| Untracked +| 29|0x0000000703a00000, 0x0000000703c00000, 0x0000000703c00000|100%| O| |TAMS 0x0000000703c00000, 0x0000000703c00000| Untracked +| 30|0x0000000703c00000, 0x0000000703e00000, 0x0000000703e00000|100%| O| |TAMS 0x0000000703e00000, 0x0000000703e00000| Untracked +| 31|0x0000000703e00000, 0x0000000704000000, 0x0000000704000000|100%| O| |TAMS 0x0000000704000000, 0x0000000704000000| Untracked +| 32|0x0000000704000000, 0x0000000704200000, 0x0000000704200000|100%| O| |TAMS 0x0000000704200000, 0x0000000704200000| Untracked +| 33|0x0000000704200000, 0x0000000704400000, 0x0000000704400000|100%| O| |TAMS 0x0000000704400000, 0x0000000704400000| Untracked +| 34|0x0000000704400000, 0x0000000704600000, 0x0000000704600000|100%| O| |TAMS 0x0000000704600000, 0x0000000704600000| Untracked +| 35|0x0000000704600000, 0x0000000704800000, 0x0000000704800000|100%| O| |TAMS 0x0000000704800000, 0x0000000704800000| Untracked +| 36|0x0000000704800000, 0x0000000704a00000, 0x0000000704a00000|100%| O| |TAMS 0x0000000704a00000, 0x0000000704a00000| Untracked +| 37|0x0000000704a00000, 0x0000000704c00000, 0x0000000704c00000|100%| O| |TAMS 0x0000000704c00000, 0x0000000704c00000| Untracked +| 38|0x0000000704c00000, 0x0000000704e00000, 0x0000000704e00000|100%| O| |TAMS 0x0000000704e00000, 0x0000000704e00000| Untracked +| 39|0x0000000704e00000, 0x0000000705000000, 0x0000000705000000|100%| O| |TAMS 0x0000000705000000, 0x0000000705000000| Untracked +| 40|0x0000000705000000, 0x0000000705200000, 0x0000000705200000|100%| O| |TAMS 0x0000000705200000, 0x0000000705200000| Untracked +| 41|0x0000000705200000, 0x0000000705400000, 0x0000000705400000|100%| O| |TAMS 0x0000000705200000, 0x0000000705400000| Untracked +| 42|0x0000000705400000, 0x0000000705600000, 0x0000000705600000|100%| O| |TAMS 0x0000000705600000, 0x0000000705600000| Untracked +| 43|0x0000000705600000, 0x0000000705800000, 0x0000000705800000|100%| O| |TAMS 0x0000000705800000, 0x0000000705800000| Untracked +| 44|0x0000000705800000, 0x0000000705a00000, 0x0000000705a00000|100%| O| |TAMS 0x0000000705a00000, 0x0000000705a00000| Untracked +| 45|0x0000000705a00000, 0x0000000705c00000, 0x0000000705c00000|100%|HS| |TAMS 0x0000000705a00000, 0x0000000705a00000| Complete +| 46|0x0000000705c00000, 0x0000000705e00000, 0x0000000705e00000|100%|HS| |TAMS 0x0000000705c00000, 0x0000000705c00000| Complete +| 47|0x0000000705e00000, 0x0000000706000000, 0x0000000706000000|100%| O| |TAMS 0x0000000706000000, 0x0000000706000000| Untracked +| 48|0x0000000706000000, 0x0000000706200000, 0x0000000706200000|100%| O| |TAMS 0x0000000706200000, 0x0000000706200000| Untracked +| 49|0x0000000706200000, 0x0000000706400000, 0x0000000706400000|100%|HS| |TAMS 0x0000000706200000, 0x0000000706200000| Complete +| 50|0x0000000706400000, 0x0000000706600000, 0x0000000706600000|100%|HS| |TAMS 0x0000000706400000, 0x0000000706400000| Complete +| 51|0x0000000706600000, 0x0000000706800000, 0x0000000706800000|100%|HS| |TAMS 0x0000000706600000, 0x0000000706600000| Complete +| 52|0x0000000706800000, 0x0000000706a00000, 0x0000000706a00000|100%|HS| |TAMS 0x0000000706800000, 0x0000000706800000| Complete +| 53|0x0000000706a00000, 0x0000000706c00000, 0x0000000706c00000|100%| O| |TAMS 0x0000000706c00000, 0x0000000706c00000| Untracked +| 54|0x0000000706c00000, 0x0000000706e00000, 0x0000000706e00000|100%| O| |TAMS 0x0000000706e00000, 0x0000000706e00000| Untracked +| 55|0x0000000706e00000, 0x0000000707000000, 0x0000000707000000|100%| O| |TAMS 0x0000000707000000, 0x0000000707000000| Untracked +| 56|0x0000000707000000, 0x0000000707200000, 0x0000000707200000|100%| O| |TAMS 0x0000000707200000, 0x0000000707200000| Untracked +| 57|0x0000000707200000, 0x0000000707400000, 0x0000000707400000|100%| O| |TAMS 0x0000000707400000, 0x0000000707400000| Untracked +| 58|0x0000000707400000, 0x0000000707600000, 0x0000000707600000|100%| O| |TAMS 0x0000000707600000, 0x0000000707600000| Untracked +| 59|0x0000000707600000, 0x0000000707800000, 0x0000000707800000|100%| O| |TAMS 0x0000000707800000, 0x0000000707800000| Untracked +| 60|0x0000000707800000, 0x0000000707a00000, 0x0000000707a00000|100%|HS| |TAMS 0x0000000707800000, 0x0000000707800000| Complete +| 61|0x0000000707a00000, 0x0000000707c00000, 0x0000000707c00000|100%|HS| |TAMS 0x0000000707a00000, 0x0000000707a00000| Complete +| 62|0x0000000707c00000, 0x0000000707e00000, 0x0000000707e00000|100%|HC| |TAMS 0x0000000707c00000, 0x0000000707c00000| Complete +| 63|0x0000000707e00000, 0x0000000708000000, 0x0000000708000000|100%|HC| |TAMS 0x0000000707e00000, 0x0000000707e00000| Complete +| 64|0x0000000708000000, 0x0000000708200000, 0x0000000708200000|100%|HS| |TAMS 0x0000000708000000, 0x0000000708000000| Complete +| 65|0x0000000708200000, 0x0000000708400000, 0x0000000708400000|100%|HS| |TAMS 0x0000000708200000, 0x0000000708200000| Complete +| 66|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| O| |TAMS 0x0000000708600000, 0x0000000708600000| Untracked +| 67|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| O| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked +| 68|0x0000000708800000, 0x0000000708a00000, 0x0000000708a00000|100%| O| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked +| 69|0x0000000708a00000, 0x0000000708c00000, 0x0000000708c00000|100%| O| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked +| 70|0x0000000708c00000, 0x0000000708e00000, 0x0000000708e00000|100%| O| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked +| 71|0x0000000708e00000, 0x0000000709000000, 0x0000000709000000|100%|HS| |TAMS 0x0000000708e00000, 0x0000000708e00000| Complete +| 72|0x0000000709000000, 0x0000000709200000, 0x0000000709200000|100%| O| |TAMS 0x00000007090e6e00, 0x0000000709200000| Untracked +| 73|0x0000000709200000, 0x0000000709400000, 0x0000000709400000|100%|HS| |TAMS 0x0000000709200000, 0x0000000709200000| Complete +| 74|0x0000000709400000, 0x0000000709600000, 0x0000000709600000|100%|HC| |TAMS 0x0000000709400000, 0x0000000709400000| Complete +| 75|0x0000000709600000, 0x0000000709800000, 0x0000000709800000|100%|HC| |TAMS 0x0000000709600000, 0x0000000709600000| Complete +| 76|0x0000000709800000, 0x0000000709a00000, 0x0000000709a00000|100%|HS| |TAMS 0x0000000709800000, 0x0000000709800000| Complete +| 77|0x0000000709a00000, 0x0000000709c00000, 0x0000000709c00000|100%|HS| |TAMS 0x0000000709a00000, 0x0000000709a00000| Complete +| 78|0x0000000709c00000, 0x0000000709e00000, 0x0000000709e00000|100%|HS| |TAMS 0x0000000709c00000, 0x0000000709c00000| Complete +| 79|0x0000000709e00000, 0x000000070a000000, 0x000000070a000000|100%|HS| |TAMS 0x0000000709e00000, 0x0000000709e00000| Complete +| 80|0x000000070a000000, 0x000000070a200000, 0x000000070a200000|100%|HS| |TAMS 0x000000070a000000, 0x000000070a000000| Complete +| 81|0x000000070a200000, 0x000000070a400000, 0x000000070a400000|100%|HS| |TAMS 0x000000070a200000, 0x000000070a200000| Complete +| 82|0x000000070a400000, 0x000000070a600000, 0x000000070a600000|100%|HS| |TAMS 0x000000070a400000, 0x000000070a400000| Complete +| 83|0x000000070a600000, 0x000000070a800000, 0x000000070a800000|100%|HS| |TAMS 0x000000070a600000, 0x000000070a600000| Complete +| 84|0x000000070a800000, 0x000000070aa00000, 0x000000070aa00000|100%|HS| |TAMS 0x000000070a800000, 0x000000070a800000| Complete +| 85|0x000000070aa00000, 0x000000070ac00000, 0x000000070ac00000|100%|HS| |TAMS 0x000000070aa00000, 0x000000070aa00000| Complete +| 86|0x000000070ac00000, 0x000000070ae00000, 0x000000070ae00000|100%|HS| |TAMS 0x000000070ac00000, 0x000000070ac00000| Complete +| 87|0x000000070ae00000, 0x000000070b000000, 0x000000070b000000|100%|HS| |TAMS 0x000000070ae00000, 0x000000070ae00000| Complete +| 88|0x000000070b000000, 0x000000070b200000, 0x000000070b200000|100%|HS| |TAMS 0x000000070b000000, 0x000000070b000000| Complete +| 89|0x000000070b200000, 0x000000070b400000, 0x000000070b400000|100%|HS| |TAMS 0x000000070b200000, 0x000000070b200000| Complete +| 90|0x000000070b400000, 0x000000070b600000, 0x000000070b600000|100%|HS| |TAMS 0x000000070b400000, 0x000000070b400000| Complete +| 91|0x000000070b600000, 0x000000070b800000, 0x000000070b800000|100%|HS| |TAMS 0x000000070b600000, 0x000000070b600000| Complete +| 92|0x000000070b800000, 0x000000070ba00000, 0x000000070ba00000|100%|HS| |TAMS 0x000000070b800000, 0x000000070b800000| Complete +| 93|0x000000070ba00000, 0x000000070bc00000, 0x000000070bc00000|100%|HS| |TAMS 0x000000070ba00000, 0x000000070ba00000| Complete +| 94|0x000000070bc00000, 0x000000070be00000, 0x000000070be00000|100%|HS| |TAMS 0x000000070bc00000, 0x000000070bc00000| Complete +| 95|0x000000070be00000, 0x000000070c000000, 0x000000070c000000|100%|HC| |TAMS 0x000000070be00000, 0x000000070be00000| Complete +| 96|0x000000070c000000, 0x000000070c200000, 0x000000070c200000|100%|HC| |TAMS 0x000000070c000000, 0x000000070c000000| Complete +| 97|0x000000070c200000, 0x000000070c400000, 0x000000070c400000|100%|HS| |TAMS 0x000000070c200000, 0x000000070c200000| Complete +| 98|0x000000070c400000, 0x000000070c600000, 0x000000070c600000|100%|HS| |TAMS 0x000000070c400000, 0x000000070c400000| Complete +| 99|0x000000070c600000, 0x000000070c800000, 0x000000070c800000|100%|HC| |TAMS 0x000000070c600000, 0x000000070c600000| Complete +| 100|0x000000070c800000, 0x000000070ca00000, 0x000000070ca00000|100%|HC| |TAMS 0x000000070c800000, 0x000000070c800000| Complete +| 101|0x000000070ca00000, 0x000000070cc00000, 0x000000070cc00000|100%|HS| |TAMS 0x000000070ca00000, 0x000000070ca00000| Complete +| 102|0x000000070cc00000, 0x000000070ce00000, 0x000000070ce00000|100%|HC| |TAMS 0x000000070cc00000, 0x000000070cc00000| Complete +| 103|0x000000070ce00000, 0x000000070d000000, 0x000000070d000000|100%|HC| |TAMS 0x000000070ce00000, 0x000000070ce00000| Complete +| 104|0x000000070d000000, 0x000000070d200000, 0x000000070d200000|100%|HS| |TAMS 0x000000070d000000, 0x000000070d000000| Complete +| 105|0x000000070d200000, 0x000000070d400000, 0x000000070d400000|100%|HS| |TAMS 0x000000070d200000, 0x000000070d200000| Complete +| 106|0x000000070d400000, 0x000000070d600000, 0x000000070d600000|100%|HC| |TAMS 0x000000070d400000, 0x000000070d400000| Complete +| 107|0x000000070d600000, 0x000000070d800000, 0x000000070d800000|100%|HC| |TAMS 0x000000070d600000, 0x000000070d600000| Complete +| 108|0x000000070d800000, 0x000000070da00000, 0x000000070da00000|100%|HS| |TAMS 0x000000070d800000, 0x000000070d800000| Complete +| 109|0x000000070da00000, 0x000000070dc00000, 0x000000070dc00000|100%|HS| |TAMS 0x000000070da00000, 0x000000070da00000| Complete +| 110|0x000000070dc00000, 0x000000070de00000, 0x000000070de00000|100%|HS| |TAMS 0x000000070dc00000, 0x000000070dc00000| Complete +| 111|0x000000070de00000, 0x000000070e000000, 0x000000070e000000|100%|HS| |TAMS 0x000000070de00000, 0x000000070de00000| Complete +| 112|0x000000070e000000, 0x000000070e200000, 0x000000070e200000|100%|HS| |TAMS 0x000000070e000000, 0x000000070e000000| Complete +| 113|0x000000070e200000, 0x000000070e400000, 0x000000070e400000|100%|HS| |TAMS 0x000000070e200000, 0x000000070e200000| Complete +| 114|0x000000070e400000, 0x000000070e600000, 0x000000070e600000|100%|HS| |TAMS 0x000000070e400000, 0x000000070e400000| Complete +| 115|0x000000070e600000, 0x000000070e800000, 0x000000070e800000|100%|HS| |TAMS 0x000000070e600000, 0x000000070e600000| Complete +| 116|0x000000070e800000, 0x000000070ea00000, 0x000000070ea00000|100%|HS| |TAMS 0x000000070e800000, 0x000000070e800000| Complete +| 117|0x000000070ea00000, 0x000000070ec00000, 0x000000070ec00000|100%|HS| |TAMS 0x000000070ea00000, 0x000000070ea00000| Complete +| 118|0x000000070ec00000, 0x000000070ee00000, 0x000000070ee00000|100%|HS| |TAMS 0x000000070ec00000, 0x000000070ec00000| Complete +| 119|0x000000070ee00000, 0x000000070f000000, 0x000000070f000000|100%|HS| |TAMS 0x000000070ee00000, 0x000000070ee00000| Complete +| 120|0x000000070f000000, 0x000000070f200000, 0x000000070f200000|100%|HS| |TAMS 0x000000070f000000, 0x000000070f000000| Complete +| 121|0x000000070f200000, 0x000000070f400000, 0x000000070f400000|100%|HS| |TAMS 0x000000070f200000, 0x000000070f200000| Complete +| 122|0x000000070f400000, 0x000000070f600000, 0x000000070f600000|100%|HS| |TAMS 0x000000070f400000, 0x000000070f400000| Complete +| 123|0x000000070f600000, 0x000000070f800000, 0x000000070f800000|100%|HS| |TAMS 0x000000070f600000, 0x000000070f600000| Complete +| 124|0x000000070f800000, 0x000000070fa00000, 0x000000070fa00000|100%|HS| |TAMS 0x000000070f800000, 0x000000070f800000| Complete +| 125|0x000000070fa00000, 0x000000070fc00000, 0x000000070fc00000|100%|HS| |TAMS 0x000000070fa00000, 0x000000070fa00000| Complete +| 126|0x000000070fc00000, 0x000000070fe00000, 0x000000070fe00000|100%|HS| |TAMS 0x000000070fc00000, 0x000000070fc00000| Complete +| 127|0x000000070fe00000, 0x0000000710000000, 0x0000000710000000|100%|HS| |TAMS 0x000000070fe00000, 0x000000070fe00000| Complete +| 128|0x0000000710000000, 0x0000000710200000, 0x0000000710200000|100%|HS| |TAMS 0x0000000710000000, 0x0000000710000000| Complete +| 129|0x0000000710200000, 0x0000000710400000, 0x0000000710400000|100%|HC| |TAMS 0x0000000710200000, 0x0000000710200000| Complete +| 130|0x0000000710400000, 0x0000000710600000, 0x0000000710600000|100%|HC| |TAMS 0x0000000710400000, 0x0000000710400000| Complete +| 131|0x0000000710600000, 0x0000000710800000, 0x0000000710800000|100%|HS| |TAMS 0x0000000710600000, 0x0000000710600000| Complete +| 132|0x0000000710800000, 0x0000000710a00000, 0x0000000710a00000|100%|HS| |TAMS 0x0000000710800000, 0x0000000710800000| Complete +| 133|0x0000000710a00000, 0x0000000710c00000, 0x0000000710c00000|100%|HC| |TAMS 0x0000000710a00000, 0x0000000710a00000| Complete +| 134|0x0000000710c00000, 0x0000000710e00000, 0x0000000710e00000|100%|HC| |TAMS 0x0000000710c00000, 0x0000000710c00000| Complete +| 135|0x0000000710e00000, 0x0000000711000000, 0x0000000711000000|100%|HS| |TAMS 0x0000000710e00000, 0x0000000710e00000| Complete +| 136|0x0000000711000000, 0x0000000711200000, 0x0000000711200000|100%|HS| |TAMS 0x0000000711000000, 0x0000000711000000| Complete +| 137|0x0000000711200000, 0x0000000711400000, 0x0000000711400000|100%|HS| |TAMS 0x0000000711200000, 0x0000000711200000| Complete +| 138|0x0000000711400000, 0x0000000711600000, 0x0000000711600000|100%|HS| |TAMS 0x0000000711400000, 0x0000000711400000| Complete +| 139|0x0000000711600000, 0x0000000711800000, 0x0000000711800000|100%|HS| |TAMS 0x0000000711600000, 0x0000000711600000| Complete +| 140|0x0000000711800000, 0x0000000711a00000, 0x0000000711a00000|100%|HS| |TAMS 0x0000000711800000, 0x0000000711800000| Complete +| 141|0x0000000711a00000, 0x0000000711c00000, 0x0000000711c00000|100%|HS| |TAMS 0x0000000711a00000, 0x0000000711a00000| Complete +| 142|0x0000000711c00000, 0x0000000711e00000, 0x0000000711e00000|100%|HS| |TAMS 0x0000000711c00000, 0x0000000711c00000| Complete +| 143|0x0000000711e00000, 0x0000000712000000, 0x0000000712000000|100%|HS| |TAMS 0x0000000711e00000, 0x0000000711e00000| Complete +| 144|0x0000000712000000, 0x0000000712200000, 0x0000000712200000|100%|HS| |TAMS 0x0000000712000000, 0x0000000712000000| Complete +| 145|0x0000000712200000, 0x0000000712200000, 0x0000000712400000| 0%| F| |TAMS 0x0000000712200000, 0x0000000712200000| Untracked +| 146|0x0000000712400000, 0x0000000712600000, 0x0000000712600000|100%| O| |TAMS 0x0000000712400000, 0x0000000712600000| Untracked +| 147|0x0000000712600000, 0x0000000712800000, 0x0000000712800000|100%| O| |TAMS 0x0000000712600000, 0x0000000712800000| Untracked +| 148|0x0000000712800000, 0x0000000712800000, 0x0000000712a00000| 0%| F| |TAMS 0x0000000712800000, 0x0000000712800000| Untracked +| 149|0x0000000712a00000, 0x0000000712a00000, 0x0000000712c00000| 0%| F| |TAMS 0x0000000712a00000, 0x0000000712a00000| Untracked +| 150|0x0000000712c00000, 0x0000000712dea400, 0x0000000712e00000| 95%| O| |TAMS 0x0000000712c00000, 0x0000000712d9d000| Untracked +| 151|0x0000000712e00000, 0x0000000713000000, 0x0000000713000000|100%| O| |TAMS 0x0000000712e00000, 0x0000000713000000| Untracked +| 152|0x0000000713000000, 0x0000000713200000, 0x0000000713200000|100%|HS| |TAMS 0x0000000713000000, 0x0000000713000000| Complete +| 153|0x0000000713200000, 0x0000000713400000, 0x0000000713400000|100%|HC| |TAMS 0x0000000713200000, 0x0000000713200000| Complete +| 154|0x0000000713400000, 0x0000000713600000, 0x0000000713600000|100%|HC| |TAMS 0x0000000713400000, 0x0000000713400000| Complete +| 155|0x0000000713600000, 0x0000000713600000, 0x0000000713800000| 0%| F| |TAMS 0x0000000713600000, 0x0000000713600000| Untracked +| 156|0x0000000713800000, 0x0000000713800000, 0x0000000713a00000| 0%| F| |TAMS 0x0000000713800000, 0x0000000713800000| Untracked +| 157|0x0000000713a00000, 0x0000000713a00000, 0x0000000713c00000| 0%| F| |TAMS 0x0000000713a00000, 0x0000000713a00000| Untracked +| 158|0x0000000713c00000, 0x0000000713c00000, 0x0000000713e00000| 0%| F| |TAMS 0x0000000713c00000, 0x0000000713c00000| Untracked +| 159|0x0000000713e00000, 0x0000000713e00000, 0x0000000714000000| 0%| F| |TAMS 0x0000000713e00000, 0x0000000713e00000| Untracked +| 160|0x0000000714000000, 0x0000000714000000, 0x0000000714200000| 0%| F| |TAMS 0x0000000714000000, 0x0000000714000000| Untracked +| 161|0x0000000714200000, 0x0000000714200000, 0x0000000714400000| 0%| F| |TAMS 0x0000000714200000, 0x0000000714200000| Untracked +| 162|0x0000000714400000, 0x0000000714400000, 0x0000000714600000| 0%| F| |TAMS 0x0000000714400000, 0x0000000714400000| Untracked +| 163|0x0000000714600000, 0x0000000714600000, 0x0000000714800000| 0%| F| |TAMS 0x0000000714600000, 0x0000000714600000| Untracked +| 164|0x0000000714800000, 0x0000000714800000, 0x0000000714a00000| 0%| F| |TAMS 0x0000000714800000, 0x0000000714800000| Untracked +| 165|0x0000000714a00000, 0x0000000714a00000, 0x0000000714c00000| 0%| F| |TAMS 0x0000000714a00000, 0x0000000714a00000| Untracked +| 166|0x0000000714c00000, 0x0000000714c00000, 0x0000000714e00000| 0%| F| |TAMS 0x0000000714c00000, 0x0000000714c00000| Untracked +| 167|0x0000000714e00000, 0x0000000714e00000, 0x0000000715000000| 0%| F| |TAMS 0x0000000714e00000, 0x0000000714e00000| Untracked +| 168|0x0000000715000000, 0x0000000715000000, 0x0000000715200000| 0%| F| |TAMS 0x0000000715000000, 0x0000000715000000| Untracked +| 169|0x0000000715200000, 0x0000000715200000, 0x0000000715400000| 0%| F| |TAMS 0x0000000715200000, 0x0000000715200000| Untracked +| 170|0x0000000715400000, 0x0000000715400000, 0x0000000715600000| 0%| F| |TAMS 0x0000000715400000, 0x0000000715400000| Untracked +| 171|0x0000000715600000, 0x0000000715600000, 0x0000000715800000| 0%| F| |TAMS 0x0000000715600000, 0x0000000715600000| Untracked +| 172|0x0000000715800000, 0x0000000715800000, 0x0000000715a00000| 0%| F| |TAMS 0x0000000715800000, 0x0000000715800000| Untracked +| 173|0x0000000715a00000, 0x0000000715a00000, 0x0000000715c00000| 0%| F| |TAMS 0x0000000715a00000, 0x0000000715a00000| Untracked +| 174|0x0000000715c00000, 0x0000000715c00000, 0x0000000715e00000| 0%| F| |TAMS 0x0000000715c00000, 0x0000000715c00000| Untracked +| 175|0x0000000715e00000, 0x0000000715e00000, 0x0000000716000000| 0%| F| |TAMS 0x0000000715e00000, 0x0000000715e00000| Untracked +| 176|0x0000000716000000, 0x0000000716000000, 0x0000000716200000| 0%| F| |TAMS 0x0000000716000000, 0x0000000716000000| Untracked +| 177|0x0000000716200000, 0x0000000716200000, 0x0000000716400000| 0%| F| |TAMS 0x0000000716200000, 0x0000000716200000| Untracked +| 178|0x0000000716400000, 0x0000000716400000, 0x0000000716600000| 0%| F| |TAMS 0x0000000716400000, 0x0000000716400000| Untracked +| 179|0x0000000716600000, 0x0000000716600000, 0x0000000716800000| 0%| F| |TAMS 0x0000000716600000, 0x0000000716600000| Untracked +| 180|0x0000000716800000, 0x0000000716800000, 0x0000000716a00000| 0%| F| |TAMS 0x0000000716800000, 0x0000000716800000| Untracked +| 181|0x0000000716a00000, 0x0000000716a00000, 0x0000000716c00000| 0%| F| |TAMS 0x0000000716a00000, 0x0000000716a00000| Untracked +| 182|0x0000000716c00000, 0x0000000716c00000, 0x0000000716e00000| 0%| F| |TAMS 0x0000000716c00000, 0x0000000716c00000| Untracked +| 183|0x0000000716e00000, 0x0000000716e00000, 0x0000000717000000| 0%| F| |TAMS 0x0000000716e00000, 0x0000000716e00000| Untracked +| 184|0x0000000717000000, 0x0000000717000000, 0x0000000717200000| 0%| F| |TAMS 0x0000000717000000, 0x0000000717000000| Untracked +| 185|0x0000000717200000, 0x0000000717200000, 0x0000000717400000| 0%| F| |TAMS 0x0000000717200000, 0x0000000717200000| Untracked +| 186|0x0000000717400000, 0x0000000717400000, 0x0000000717600000| 0%| F| |TAMS 0x0000000717400000, 0x0000000717400000| Untracked +| 187|0x0000000717600000, 0x0000000717600000, 0x0000000717800000| 0%| F| |TAMS 0x0000000717600000, 0x0000000717600000| Untracked +| 188|0x0000000717800000, 0x0000000717800000, 0x0000000717a00000| 0%| F| |TAMS 0x0000000717800000, 0x0000000717800000| Untracked +| 189|0x0000000717a00000, 0x0000000717a00000, 0x0000000717c00000| 0%| F| |TAMS 0x0000000717a00000, 0x0000000717a00000| Untracked +| 190|0x0000000717c00000, 0x0000000717c00000, 0x0000000717e00000| 0%| F| |TAMS 0x0000000717c00000, 0x0000000717c00000| Untracked +| 191|0x0000000717e00000, 0x0000000717e00000, 0x0000000718000000| 0%| F| |TAMS 0x0000000717e00000, 0x0000000717e00000| Untracked +| 192|0x0000000718000000, 0x0000000718000000, 0x0000000718200000| 0%| F| |TAMS 0x0000000718000000, 0x0000000718000000| Untracked +| 193|0x0000000718200000, 0x0000000718200000, 0x0000000718400000| 0%| F| |TAMS 0x0000000718200000, 0x0000000718200000| Untracked +| 194|0x0000000718400000, 0x0000000718400000, 0x0000000718600000| 0%| F| |TAMS 0x0000000718400000, 0x0000000718400000| Untracked +| 195|0x0000000718600000, 0x0000000718600000, 0x0000000718800000| 0%| F| |TAMS 0x0000000718600000, 0x0000000718600000| Untracked +| 196|0x0000000718800000, 0x0000000718800000, 0x0000000718a00000| 0%| F| |TAMS 0x0000000718800000, 0x0000000718800000| Untracked +| 197|0x0000000718a00000, 0x0000000718a00000, 0x0000000718c00000| 0%| F| |TAMS 0x0000000718a00000, 0x0000000718a00000| Untracked +| 198|0x0000000718c00000, 0x0000000718c00000, 0x0000000718e00000| 0%| F| |TAMS 0x0000000718c00000, 0x0000000718c00000| Untracked +| 199|0x0000000718e00000, 0x0000000718e00000, 0x0000000719000000| 0%| F| |TAMS 0x0000000718e00000, 0x0000000718e00000| Untracked +| 200|0x0000000719000000, 0x0000000719000000, 0x0000000719200000| 0%| F| |TAMS 0x0000000719000000, 0x0000000719000000| Untracked +| 201|0x0000000719200000, 0x0000000719200000, 0x0000000719400000| 0%| F| |TAMS 0x0000000719200000, 0x0000000719200000| Untracked +| 202|0x0000000719400000, 0x0000000719400000, 0x0000000719600000| 0%| F| |TAMS 0x0000000719400000, 0x0000000719400000| Untracked +| 203|0x0000000719600000, 0x0000000719600000, 0x0000000719800000| 0%| F| |TAMS 0x0000000719600000, 0x0000000719600000| Untracked +| 204|0x0000000719800000, 0x0000000719800000, 0x0000000719a00000| 0%| F| |TAMS 0x0000000719800000, 0x0000000719800000| Untracked +| 205|0x0000000719a00000, 0x0000000719a00000, 0x0000000719c00000| 0%| F| |TAMS 0x0000000719a00000, 0x0000000719a00000| Untracked +| 206|0x0000000719c00000, 0x0000000719c00000, 0x0000000719e00000| 0%| F| |TAMS 0x0000000719c00000, 0x0000000719c00000| Untracked +| 207|0x0000000719e00000, 0x0000000719e00000, 0x000000071a000000| 0%| F| |TAMS 0x0000000719e00000, 0x0000000719e00000| Untracked +| 208|0x000000071a000000, 0x000000071a000000, 0x000000071a200000| 0%| F| |TAMS 0x000000071a000000, 0x000000071a000000| Untracked +| 209|0x000000071a200000, 0x000000071a200000, 0x000000071a400000| 0%| F| |TAMS 0x000000071a200000, 0x000000071a200000| Untracked +| 210|0x000000071a400000, 0x000000071a400000, 0x000000071a600000| 0%| F| |TAMS 0x000000071a400000, 0x000000071a400000| Untracked +| 211|0x000000071a600000, 0x000000071a600000, 0x000000071a800000| 0%| F| |TAMS 0x000000071a600000, 0x000000071a600000| Untracked +| 212|0x000000071a800000, 0x000000071a800000, 0x000000071aa00000| 0%| F| |TAMS 0x000000071a800000, 0x000000071a800000| Untracked +| 213|0x000000071aa00000, 0x000000071aa00000, 0x000000071ac00000| 0%| F| |TAMS 0x000000071aa00000, 0x000000071aa00000| Untracked +| 214|0x000000071ac00000, 0x000000071ac00000, 0x000000071ae00000| 0%| F| |TAMS 0x000000071ac00000, 0x000000071ac00000| Untracked +| 215|0x000000071ae00000, 0x000000071ae00000, 0x000000071b000000| 0%| F| |TAMS 0x000000071ae00000, 0x000000071ae00000| Untracked +| 216|0x000000071b000000, 0x000000071b000000, 0x000000071b200000| 0%| F| |TAMS 0x000000071b000000, 0x000000071b000000| Untracked +| 217|0x000000071b200000, 0x000000071b200000, 0x000000071b400000| 0%| F| |TAMS 0x000000071b200000, 0x000000071b200000| Untracked +| 218|0x000000071b400000, 0x000000071b400000, 0x000000071b600000| 0%| F| |TAMS 0x000000071b400000, 0x000000071b400000| Untracked +| 219|0x000000071b600000, 0x000000071b600000, 0x000000071b800000| 0%| F| |TAMS 0x000000071b600000, 0x000000071b600000| Untracked +| 220|0x000000071b800000, 0x000000071b800000, 0x000000071ba00000| 0%| F| |TAMS 0x000000071b800000, 0x000000071b800000| Untracked +| 221|0x000000071ba00000, 0x000000071ba00000, 0x000000071bc00000| 0%| F| |TAMS 0x000000071ba00000, 0x000000071ba00000| Untracked +| 222|0x000000071bc00000, 0x000000071bc00000, 0x000000071be00000| 0%| F| |TAMS 0x000000071bc00000, 0x000000071bc00000| Untracked +| 223|0x000000071be00000, 0x000000071be00000, 0x000000071c000000| 0%| F| |TAMS 0x000000071be00000, 0x000000071be00000| Untracked +| 224|0x000000071c000000, 0x000000071c000000, 0x000000071c200000| 0%| F| |TAMS 0x000000071c000000, 0x000000071c000000| Untracked +| 225|0x000000071c200000, 0x000000071c200000, 0x000000071c400000| 0%| F| |TAMS 0x000000071c200000, 0x000000071c200000| Untracked +| 226|0x000000071c400000, 0x000000071c400000, 0x000000071c600000| 0%| F| |TAMS 0x000000071c400000, 0x000000071c400000| Untracked +| 227|0x000000071c600000, 0x000000071c600000, 0x000000071c800000| 0%| F| |TAMS 0x000000071c600000, 0x000000071c600000| Untracked +| 228|0x000000071c800000, 0x000000071c800000, 0x000000071ca00000| 0%| F| |TAMS 0x000000071c800000, 0x000000071c800000| Untracked +| 229|0x000000071ca00000, 0x000000071ca00000, 0x000000071cc00000| 0%| F| |TAMS 0x000000071ca00000, 0x000000071ca00000| Untracked +| 230|0x000000071cc00000, 0x000000071cc00000, 0x000000071ce00000| 0%| F| |TAMS 0x000000071cc00000, 0x000000071cc00000| Untracked +| 231|0x000000071ce00000, 0x000000071ce00000, 0x000000071d000000| 0%| F| |TAMS 0x000000071ce00000, 0x000000071ce00000| Untracked +| 232|0x000000071d000000, 0x000000071d000000, 0x000000071d200000| 0%| F| |TAMS 0x000000071d000000, 0x000000071d000000| Untracked +| 233|0x000000071d200000, 0x000000071d200000, 0x000000071d400000| 0%| F| |TAMS 0x000000071d200000, 0x000000071d200000| Untracked +| 234|0x000000071d400000, 0x000000071d400000, 0x000000071d600000| 0%| F| |TAMS 0x000000071d400000, 0x000000071d400000| Untracked +| 235|0x000000071d600000, 0x000000071d600000, 0x000000071d800000| 0%| F| |TAMS 0x000000071d600000, 0x000000071d600000| Untracked +| 236|0x000000071d800000, 0x000000071d800000, 0x000000071da00000| 0%| F| |TAMS 0x000000071d800000, 0x000000071d800000| Untracked +| 237|0x000000071da00000, 0x000000071da00000, 0x000000071dc00000| 0%| F| |TAMS 0x000000071da00000, 0x000000071da00000| Untracked +| 238|0x000000071dc00000, 0x000000071dc00000, 0x000000071de00000| 0%| F| |TAMS 0x000000071dc00000, 0x000000071dc00000| Untracked +| 239|0x000000071de00000, 0x000000071de00000, 0x000000071e000000| 0%| F| |TAMS 0x000000071de00000, 0x000000071de00000| Untracked +| 240|0x000000071e000000, 0x000000071e000000, 0x000000071e200000| 0%| F| |TAMS 0x000000071e000000, 0x000000071e000000| Untracked +| 241|0x000000071e200000, 0x000000071e200000, 0x000000071e400000| 0%| F| |TAMS 0x000000071e200000, 0x000000071e200000| Untracked +| 242|0x000000071e400000, 0x000000071e400000, 0x000000071e600000| 0%| F| |TAMS 0x000000071e400000, 0x000000071e400000| Untracked +| 243|0x000000071e600000, 0x000000071e600000, 0x000000071e800000| 0%| F| |TAMS 0x000000071e600000, 0x000000071e600000| Untracked +| 244|0x000000071e800000, 0x000000071e800000, 0x000000071ea00000| 0%| F| |TAMS 0x000000071e800000, 0x000000071e800000| Untracked +| 245|0x000000071ea00000, 0x000000071ea00000, 0x000000071ec00000| 0%| F| |TAMS 0x000000071ea00000, 0x000000071ea00000| Untracked +| 246|0x000000071ec00000, 0x000000071ec00000, 0x000000071ee00000| 0%| F| |TAMS 0x000000071ec00000, 0x000000071ec00000| Untracked +| 247|0x000000071ee00000, 0x000000071ee00000, 0x000000071f000000| 0%| F| |TAMS 0x000000071ee00000, 0x000000071ee00000| Untracked +| 248|0x000000071f000000, 0x000000071f000000, 0x000000071f200000| 0%| F| |TAMS 0x000000071f000000, 0x000000071f000000| Untracked +| 249|0x000000071f200000, 0x000000071f200000, 0x000000071f400000| 0%| F| |TAMS 0x000000071f200000, 0x000000071f200000| Untracked +| 250|0x000000071f400000, 0x000000071f400000, 0x000000071f600000| 0%| F| |TAMS 0x000000071f400000, 0x000000071f400000| Untracked +| 251|0x000000071f600000, 0x000000071f600000, 0x000000071f800000| 0%| F| |TAMS 0x000000071f600000, 0x000000071f600000| Untracked +| 252|0x000000071f800000, 0x000000071f800000, 0x000000071fa00000| 0%| F| |TAMS 0x000000071f800000, 0x000000071f800000| Untracked +| 253|0x000000071fa00000, 0x000000071fa00000, 0x000000071fc00000| 0%| F| |TAMS 0x000000071fa00000, 0x000000071fa00000| Untracked +| 254|0x000000071fc00000, 0x000000071fc00000, 0x000000071fe00000| 0%| F| |TAMS 0x000000071fc00000, 0x000000071fc00000| Untracked +| 255|0x000000071fe00000, 0x000000071fe00000, 0x0000000720000000| 0%| F| |TAMS 0x000000071fe00000, 0x000000071fe00000| Untracked +| 256|0x0000000720000000, 0x0000000720000000, 0x0000000720200000| 0%| F| |TAMS 0x0000000720000000, 0x0000000720000000| Untracked +| 257|0x0000000720200000, 0x0000000720200000, 0x0000000720400000| 0%| F| |TAMS 0x0000000720200000, 0x0000000720200000| Untracked +| 258|0x0000000720400000, 0x0000000720400000, 0x0000000720600000| 0%| F| |TAMS 0x0000000720400000, 0x0000000720400000| Untracked +| 259|0x0000000720600000, 0x0000000720600000, 0x0000000720800000| 0%| F| |TAMS 0x0000000720600000, 0x0000000720600000| Untracked +| 260|0x0000000720800000, 0x0000000720800000, 0x0000000720a00000| 0%| F| |TAMS 0x0000000720800000, 0x0000000720800000| Untracked +| 261|0x0000000720a00000, 0x0000000720a00000, 0x0000000720c00000| 0%| F| |TAMS 0x0000000720a00000, 0x0000000720a00000| Untracked +| 262|0x0000000720c00000, 0x0000000720c00000, 0x0000000720e00000| 0%| F| |TAMS 0x0000000720c00000, 0x0000000720c00000| Untracked +| 263|0x0000000720e00000, 0x0000000720e00000, 0x0000000721000000| 0%| F| |TAMS 0x0000000720e00000, 0x0000000720e00000| Untracked +| 264|0x0000000721000000, 0x0000000721000000, 0x0000000721200000| 0%| F| |TAMS 0x0000000721000000, 0x0000000721000000| Untracked +| 265|0x0000000721200000, 0x0000000721200000, 0x0000000721400000| 0%| F| |TAMS 0x0000000721200000, 0x0000000721200000| Untracked +| 266|0x0000000721400000, 0x0000000721400000, 0x0000000721600000| 0%| F| |TAMS 0x0000000721400000, 0x0000000721400000| Untracked +| 267|0x0000000721600000, 0x0000000721600000, 0x0000000721800000| 0%| F| |TAMS 0x0000000721600000, 0x0000000721600000| Untracked +| 268|0x0000000721800000, 0x0000000721800000, 0x0000000721a00000| 0%| F| |TAMS 0x0000000721800000, 0x0000000721800000| Untracked +| 269|0x0000000721a00000, 0x0000000721a00000, 0x0000000721c00000| 0%| F| |TAMS 0x0000000721a00000, 0x0000000721a00000| Untracked +| 270|0x0000000721c00000, 0x0000000721c00000, 0x0000000721e00000| 0%| F| |TAMS 0x0000000721c00000, 0x0000000721c00000| Untracked +| 271|0x0000000721e00000, 0x0000000721e00000, 0x0000000722000000| 0%| F| |TAMS 0x0000000721e00000, 0x0000000721e00000| Untracked +| 272|0x0000000722000000, 0x0000000722000000, 0x0000000722200000| 0%| F| |TAMS 0x0000000722000000, 0x0000000722000000| Untracked +| 273|0x0000000722200000, 0x0000000722200000, 0x0000000722400000| 0%| F| |TAMS 0x0000000722200000, 0x0000000722200000| Untracked +| 274|0x0000000722400000, 0x0000000722400000, 0x0000000722600000| 0%| F| |TAMS 0x0000000722400000, 0x0000000722400000| Untracked +| 275|0x0000000722600000, 0x0000000722600000, 0x0000000722800000| 0%| F| |TAMS 0x0000000722600000, 0x0000000722600000| Untracked +| 276|0x0000000722800000, 0x0000000722800000, 0x0000000722a00000| 0%| F| |TAMS 0x0000000722800000, 0x0000000722800000| Untracked +| 277|0x0000000722a00000, 0x0000000722a00000, 0x0000000722c00000| 0%| F| |TAMS 0x0000000722a00000, 0x0000000722a00000| Untracked +| 278|0x0000000722c00000, 0x0000000722c00000, 0x0000000722e00000| 0%| F| |TAMS 0x0000000722c00000, 0x0000000722c00000| Untracked +| 279|0x0000000722e00000, 0x0000000722e00000, 0x0000000723000000| 0%| F| |TAMS 0x0000000722e00000, 0x0000000722e00000| Untracked +| 280|0x0000000723000000, 0x0000000723000000, 0x0000000723200000| 0%| F| |TAMS 0x0000000723000000, 0x0000000723000000| Untracked +| 281|0x0000000723200000, 0x0000000723200000, 0x0000000723400000| 0%| F| |TAMS 0x0000000723200000, 0x0000000723200000| Untracked +| 282|0x0000000723400000, 0x0000000723400000, 0x0000000723600000| 0%| F| |TAMS 0x0000000723400000, 0x0000000723400000| Untracked +| 283|0x0000000723600000, 0x0000000723600000, 0x0000000723800000| 0%| F| |TAMS 0x0000000723600000, 0x0000000723600000| Untracked +| 284|0x0000000723800000, 0x0000000723800000, 0x0000000723a00000| 0%| F| |TAMS 0x0000000723800000, 0x0000000723800000| Untracked +| 285|0x0000000723a00000, 0x0000000723a00000, 0x0000000723c00000| 0%| F| |TAMS 0x0000000723a00000, 0x0000000723a00000| Untracked +| 286|0x0000000723c00000, 0x0000000723c00000, 0x0000000723e00000| 0%| F| |TAMS 0x0000000723c00000, 0x0000000723c00000| Untracked +| 287|0x0000000723e00000, 0x0000000723e00000, 0x0000000724000000| 0%| F| |TAMS 0x0000000723e00000, 0x0000000723e00000| Untracked +| 288|0x0000000724000000, 0x0000000724000000, 0x0000000724200000| 0%| F| |TAMS 0x0000000724000000, 0x0000000724000000| Untracked +| 289|0x0000000724200000, 0x0000000724200000, 0x0000000724400000| 0%| F| |TAMS 0x0000000724200000, 0x0000000724200000| Untracked +| 290|0x0000000724400000, 0x0000000724400000, 0x0000000724600000| 0%| F| |TAMS 0x0000000724400000, 0x0000000724400000| Untracked +| 291|0x0000000724600000, 0x0000000724600000, 0x0000000724800000| 0%| F| |TAMS 0x0000000724600000, 0x0000000724600000| Untracked +| 292|0x0000000724800000, 0x0000000724800000, 0x0000000724a00000| 0%| F| |TAMS 0x0000000724800000, 0x0000000724800000| Untracked +| 293|0x0000000724a00000, 0x0000000724a00000, 0x0000000724c00000| 0%| F| |TAMS 0x0000000724a00000, 0x0000000724a00000| Untracked +| 294|0x0000000724c00000, 0x0000000724c00000, 0x0000000724e00000| 0%| F| |TAMS 0x0000000724c00000, 0x0000000724c00000| Untracked +| 295|0x0000000724e00000, 0x0000000724e00000, 0x0000000725000000| 0%| F| |TAMS 0x0000000724e00000, 0x0000000724e00000| Untracked +| 296|0x0000000725000000, 0x0000000725000000, 0x0000000725200000| 0%| F| |TAMS 0x0000000725000000, 0x0000000725000000| Untracked +| 297|0x0000000725200000, 0x0000000725200000, 0x0000000725400000| 0%| F| |TAMS 0x0000000725200000, 0x0000000725200000| Untracked +| 298|0x0000000725400000, 0x0000000725400000, 0x0000000725600000| 0%| F| |TAMS 0x0000000725400000, 0x0000000725400000| Untracked +| 299|0x0000000725600000, 0x0000000725600000, 0x0000000725800000| 0%| F| |TAMS 0x0000000725600000, 0x0000000725600000| Untracked +| 300|0x0000000725800000, 0x0000000725800000, 0x0000000725a00000| 0%| F| |TAMS 0x0000000725800000, 0x0000000725800000| Untracked +| 301|0x0000000725a00000, 0x0000000725a00000, 0x0000000725c00000| 0%| F| |TAMS 0x0000000725a00000, 0x0000000725a00000| Untracked +| 302|0x0000000725c00000, 0x0000000725c00000, 0x0000000725e00000| 0%| F| |TAMS 0x0000000725c00000, 0x0000000725c00000| Untracked +| 303|0x0000000725e00000, 0x0000000725e00000, 0x0000000726000000| 0%| F| |TAMS 0x0000000725e00000, 0x0000000725e00000| Untracked +| 304|0x0000000726000000, 0x0000000726000000, 0x0000000726200000| 0%| F| |TAMS 0x0000000726000000, 0x0000000726000000| Untracked +| 305|0x0000000726200000, 0x0000000726200000, 0x0000000726400000| 0%| F| |TAMS 0x0000000726200000, 0x0000000726200000| Untracked +| 306|0x0000000726400000, 0x0000000726400000, 0x0000000726600000| 0%| F| |TAMS 0x0000000726400000, 0x0000000726400000| Untracked +| 307|0x0000000726600000, 0x0000000726600000, 0x0000000726800000| 0%| F| |TAMS 0x0000000726600000, 0x0000000726600000| Untracked +| 308|0x0000000726800000, 0x0000000726800000, 0x0000000726a00000| 0%| F| |TAMS 0x0000000726800000, 0x0000000726800000| Untracked +| 309|0x0000000726a00000, 0x0000000726a00000, 0x0000000726c00000| 0%| F| |TAMS 0x0000000726a00000, 0x0000000726a00000| Untracked +| 310|0x0000000726c00000, 0x0000000726c00000, 0x0000000726e00000| 0%| F| |TAMS 0x0000000726c00000, 0x0000000726c00000| Untracked +| 311|0x0000000726e00000, 0x0000000726e00000, 0x0000000727000000| 0%| F| |TAMS 0x0000000726e00000, 0x0000000726e00000| Untracked +| 312|0x0000000727000000, 0x0000000727000000, 0x0000000727200000| 0%| F| |TAMS 0x0000000727000000, 0x0000000727000000| Untracked +| 313|0x0000000727200000, 0x0000000727200000, 0x0000000727400000| 0%| F| |TAMS 0x0000000727200000, 0x0000000727200000| Untracked +| 314|0x0000000727400000, 0x0000000727400000, 0x0000000727600000| 0%| F| |TAMS 0x0000000727400000, 0x0000000727400000| Untracked +| 315|0x0000000727600000, 0x0000000727600000, 0x0000000727800000| 0%| F| |TAMS 0x0000000727600000, 0x0000000727600000| Untracked +| 316|0x0000000727800000, 0x0000000727800000, 0x0000000727a00000| 0%| F| |TAMS 0x0000000727800000, 0x0000000727800000| Untracked +| 317|0x0000000727a00000, 0x0000000727a00000, 0x0000000727c00000| 0%| F| |TAMS 0x0000000727a00000, 0x0000000727a00000| Untracked +| 318|0x0000000727c00000, 0x0000000727c00000, 0x0000000727e00000| 0%| F| |TAMS 0x0000000727c00000, 0x0000000727c00000| Untracked +| 319|0x0000000727e00000, 0x0000000727e00000, 0x0000000728000000| 0%| F| |TAMS 0x0000000727e00000, 0x0000000727e00000| Untracked +| 320|0x0000000728000000, 0x0000000728000000, 0x0000000728200000| 0%| F| |TAMS 0x0000000728000000, 0x0000000728000000| Untracked +| 321|0x0000000728200000, 0x0000000728200000, 0x0000000728400000| 0%| F| |TAMS 0x0000000728200000, 0x0000000728200000| Untracked +| 322|0x0000000728400000, 0x0000000728400000, 0x0000000728600000| 0%| F| |TAMS 0x0000000728400000, 0x0000000728400000| Untracked +| 323|0x0000000728600000, 0x0000000728600000, 0x0000000728800000| 0%| F| |TAMS 0x0000000728600000, 0x0000000728600000| Untracked +| 324|0x0000000728800000, 0x0000000728800000, 0x0000000728a00000| 0%| F| |TAMS 0x0000000728800000, 0x0000000728800000| Untracked +| 325|0x0000000728a00000, 0x0000000728a00000, 0x0000000728c00000| 0%| F| |TAMS 0x0000000728a00000, 0x0000000728a00000| Untracked +| 326|0x0000000728c00000, 0x0000000728c00000, 0x0000000728e00000| 0%| F| |TAMS 0x0000000728c00000, 0x0000000728c00000| Untracked +| 327|0x0000000728e00000, 0x0000000728e00000, 0x0000000729000000| 0%| F| |TAMS 0x0000000728e00000, 0x0000000728e00000| Untracked +| 328|0x0000000729000000, 0x0000000729000000, 0x0000000729200000| 0%| F| |TAMS 0x0000000729000000, 0x0000000729000000| Untracked +| 329|0x0000000729200000, 0x0000000729200000, 0x0000000729400000| 0%| F| |TAMS 0x0000000729200000, 0x0000000729200000| Untracked +| 330|0x0000000729400000, 0x0000000729400000, 0x0000000729600000| 0%| F| |TAMS 0x0000000729400000, 0x0000000729400000| Untracked +| 331|0x0000000729600000, 0x0000000729600000, 0x0000000729800000| 0%| F| |TAMS 0x0000000729600000, 0x0000000729600000| Untracked +| 332|0x0000000729800000, 0x0000000729800000, 0x0000000729a00000| 0%| F| |TAMS 0x0000000729800000, 0x0000000729800000| Untracked +| 333|0x0000000729a00000, 0x0000000729a00000, 0x0000000729c00000| 0%| F| |TAMS 0x0000000729a00000, 0x0000000729a00000| Untracked +| 334|0x0000000729c00000, 0x0000000729c00000, 0x0000000729e00000| 0%| F| |TAMS 0x0000000729c00000, 0x0000000729c00000| Untracked +| 335|0x0000000729e00000, 0x0000000729e00000, 0x000000072a000000| 0%| F| |TAMS 0x0000000729e00000, 0x0000000729e00000| Untracked +| 336|0x000000072a000000, 0x000000072a000000, 0x000000072a200000| 0%| F| |TAMS 0x000000072a000000, 0x000000072a000000| Untracked +| 337|0x000000072a200000, 0x000000072a200000, 0x000000072a400000| 0%| F| |TAMS 0x000000072a200000, 0x000000072a200000| Untracked +| 338|0x000000072a400000, 0x000000072a400000, 0x000000072a600000| 0%| F| |TAMS 0x000000072a400000, 0x000000072a400000| Untracked +| 339|0x000000072a600000, 0x000000072a600000, 0x000000072a800000| 0%| F| |TAMS 0x000000072a600000, 0x000000072a600000| Untracked +| 340|0x000000072a800000, 0x000000072a800000, 0x000000072aa00000| 0%| F| |TAMS 0x000000072a800000, 0x000000072a800000| Untracked +| 341|0x000000072aa00000, 0x000000072aa00000, 0x000000072ac00000| 0%| F| |TAMS 0x000000072aa00000, 0x000000072aa00000| Untracked +| 342|0x000000072ac00000, 0x000000072ac00000, 0x000000072ae00000| 0%| F| |TAMS 0x000000072ac00000, 0x000000072ac00000| Untracked +| 343|0x000000072ae00000, 0x000000072ae00000, 0x000000072b000000| 0%| F| |TAMS 0x000000072ae00000, 0x000000072ae00000| Untracked +| 344|0x000000072b000000, 0x000000072b000000, 0x000000072b200000| 0%| F| |TAMS 0x000000072b000000, 0x000000072b000000| Untracked +| 345|0x000000072b200000, 0x000000072b200000, 0x000000072b400000| 0%| F| |TAMS 0x000000072b200000, 0x000000072b200000| Untracked +| 346|0x000000072b400000, 0x000000072b400000, 0x000000072b600000| 0%| F| |TAMS 0x000000072b400000, 0x000000072b400000| Untracked +| 347|0x000000072b600000, 0x000000072b600000, 0x000000072b800000| 0%| F| |TAMS 0x000000072b600000, 0x000000072b600000| Untracked +| 348|0x000000072b800000, 0x000000072b800000, 0x000000072ba00000| 0%| F| |TAMS 0x000000072b800000, 0x000000072b800000| Untracked +| 349|0x000000072ba00000, 0x000000072ba00000, 0x000000072bc00000| 0%| F| |TAMS 0x000000072ba00000, 0x000000072ba00000| Untracked +| 350|0x000000072bc00000, 0x000000072bc00000, 0x000000072be00000| 0%| F| |TAMS 0x000000072bc00000, 0x000000072bc00000| Untracked +| 351|0x000000072be00000, 0x000000072be00000, 0x000000072c000000| 0%| F| |TAMS 0x000000072be00000, 0x000000072be00000| Untracked +| 352|0x000000072c000000, 0x000000072c000000, 0x000000072c200000| 0%| F| |TAMS 0x000000072c000000, 0x000000072c000000| Untracked +| 353|0x000000072c200000, 0x000000072c200000, 0x000000072c400000| 0%| F| |TAMS 0x000000072c200000, 0x000000072c200000| Untracked +| 354|0x000000072c400000, 0x000000072c400000, 0x000000072c600000| 0%| F| |TAMS 0x000000072c400000, 0x000000072c400000| Untracked +| 355|0x000000072c600000, 0x000000072c600000, 0x000000072c800000| 0%| F| |TAMS 0x000000072c600000, 0x000000072c600000| Untracked +| 356|0x000000072c800000, 0x000000072c800000, 0x000000072ca00000| 0%| F| |TAMS 0x000000072c800000, 0x000000072c800000| Untracked +| 357|0x000000072ca00000, 0x000000072ca00000, 0x000000072cc00000| 0%| F| |TAMS 0x000000072ca00000, 0x000000072ca00000| Untracked +| 358|0x000000072cc00000, 0x000000072cc00000, 0x000000072ce00000| 0%| F| |TAMS 0x000000072cc00000, 0x000000072cc00000| Untracked +| 359|0x000000072ce00000, 0x000000072ce00000, 0x000000072d000000| 0%| F| |TAMS 0x000000072ce00000, 0x000000072ce00000| Untracked +| 360|0x000000072d000000, 0x000000072d000000, 0x000000072d200000| 0%| F| |TAMS 0x000000072d000000, 0x000000072d000000| Untracked +| 361|0x000000072d200000, 0x000000072d200000, 0x000000072d400000| 0%| F| |TAMS 0x000000072d200000, 0x000000072d200000| Untracked +| 362|0x000000072d400000, 0x000000072d400000, 0x000000072d600000| 0%| F| |TAMS 0x000000072d400000, 0x000000072d400000| Untracked +| 363|0x000000072d600000, 0x000000072d600000, 0x000000072d800000| 0%| F| |TAMS 0x000000072d600000, 0x000000072d600000| Untracked +| 364|0x000000072d800000, 0x000000072d800000, 0x000000072da00000| 0%| F| |TAMS 0x000000072d800000, 0x000000072d800000| Untracked +| 365|0x000000072da00000, 0x000000072da00000, 0x000000072dc00000| 0%| F| |TAMS 0x000000072da00000, 0x000000072da00000| Untracked +| 366|0x000000072dc00000, 0x000000072dc00000, 0x000000072de00000| 0%| F| |TAMS 0x000000072dc00000, 0x000000072dc00000| Untracked +| 367|0x000000072de00000, 0x000000072de00000, 0x000000072e000000| 0%| F| |TAMS 0x000000072de00000, 0x000000072de00000| Untracked +| 368|0x000000072e000000, 0x000000072e000000, 0x000000072e200000| 0%| F| |TAMS 0x000000072e000000, 0x000000072e000000| Untracked +| 369|0x000000072e200000, 0x000000072e200000, 0x000000072e400000| 0%| F| |TAMS 0x000000072e200000, 0x000000072e200000| Untracked +| 370|0x000000072e400000, 0x000000072e400000, 0x000000072e600000| 0%| F| |TAMS 0x000000072e400000, 0x000000072e400000| Untracked +| 371|0x000000072e600000, 0x000000072e600000, 0x000000072e800000| 0%| F| |TAMS 0x000000072e600000, 0x000000072e600000| Untracked +| 372|0x000000072e800000, 0x000000072e800000, 0x000000072ea00000| 0%| F| |TAMS 0x000000072e800000, 0x000000072e800000| Untracked +| 373|0x000000072ea00000, 0x000000072ea00000, 0x000000072ec00000| 0%| F| |TAMS 0x000000072ea00000, 0x000000072ea00000| Untracked +| 374|0x000000072ec00000, 0x000000072ec00000, 0x000000072ee00000| 0%| F| |TAMS 0x000000072ec00000, 0x000000072ec00000| Untracked +| 375|0x000000072ee00000, 0x000000072ee00000, 0x000000072f000000| 0%| F| |TAMS 0x000000072ee00000, 0x000000072ee00000| Untracked +| 376|0x000000072f000000, 0x000000072f000000, 0x000000072f200000| 0%| F| |TAMS 0x000000072f000000, 0x000000072f000000| Untracked +| 377|0x000000072f200000, 0x000000072f200000, 0x000000072f400000| 0%| F| |TAMS 0x000000072f200000, 0x000000072f200000| Untracked +| 378|0x000000072f400000, 0x000000072f400000, 0x000000072f600000| 0%| F| |TAMS 0x000000072f400000, 0x000000072f400000| Untracked +| 379|0x000000072f600000, 0x000000072f600000, 0x000000072f800000| 0%| F| |TAMS 0x000000072f600000, 0x000000072f600000| Untracked +| 380|0x000000072f800000, 0x000000072f800000, 0x000000072fa00000| 0%| F| |TAMS 0x000000072f800000, 0x000000072f800000| Untracked +| 381|0x000000072fa00000, 0x000000072fa00000, 0x000000072fc00000| 0%| F| |TAMS 0x000000072fa00000, 0x000000072fa00000| Untracked +| 382|0x000000072fc00000, 0x000000072fc00000, 0x000000072fe00000| 0%| F| |TAMS 0x000000072fc00000, 0x000000072fc00000| Untracked +| 383|0x000000072fe00000, 0x000000072fe00000, 0x0000000730000000| 0%| F| |TAMS 0x000000072fe00000, 0x000000072fe00000| Untracked +| 384|0x0000000730000000, 0x0000000730000000, 0x0000000730200000| 0%| F| |TAMS 0x0000000730000000, 0x0000000730000000| Untracked +| 385|0x0000000730200000, 0x0000000730200000, 0x0000000730400000| 0%| F| |TAMS 0x0000000730200000, 0x0000000730200000| Untracked +| 386|0x0000000730400000, 0x0000000730400000, 0x0000000730600000| 0%| F| |TAMS 0x0000000730400000, 0x0000000730400000| Untracked +| 387|0x0000000730600000, 0x0000000730600000, 0x0000000730800000| 0%| F| |TAMS 0x0000000730600000, 0x0000000730600000| Untracked +| 388|0x0000000730800000, 0x0000000730800000, 0x0000000730a00000| 0%| F| |TAMS 0x0000000730800000, 0x0000000730800000| Untracked +| 389|0x0000000730a00000, 0x0000000730a00000, 0x0000000730c00000| 0%| F| |TAMS 0x0000000730a00000, 0x0000000730a00000| Untracked +| 390|0x0000000730c00000, 0x0000000730c00000, 0x0000000730e00000| 0%| F| |TAMS 0x0000000730c00000, 0x0000000730c00000| Untracked +| 391|0x0000000730e00000, 0x0000000730e00000, 0x0000000731000000| 0%| F| |TAMS 0x0000000730e00000, 0x0000000730e00000| Untracked +| 392|0x0000000731000000, 0x0000000731000000, 0x0000000731200000| 0%| F| |TAMS 0x0000000731000000, 0x0000000731000000| Untracked +| 393|0x0000000731200000, 0x0000000731200000, 0x0000000731400000| 0%| F| |TAMS 0x0000000731200000, 0x0000000731200000| Untracked +| 394|0x0000000731400000, 0x0000000731400000, 0x0000000731600000| 0%| F| |TAMS 0x0000000731400000, 0x0000000731400000| Untracked +| 395|0x0000000731600000, 0x0000000731600000, 0x0000000731800000| 0%| F| |TAMS 0x0000000731600000, 0x0000000731600000| Untracked +| 396|0x0000000731800000, 0x0000000731800000, 0x0000000731a00000| 0%| F| |TAMS 0x0000000731800000, 0x0000000731800000| Untracked +| 397|0x0000000731a00000, 0x0000000731a00000, 0x0000000731c00000| 0%| F| |TAMS 0x0000000731a00000, 0x0000000731a00000| Untracked +| 398|0x0000000731c00000, 0x0000000731c9bf20, 0x0000000731e00000| 30%| S|CS|TAMS 0x0000000731c00000, 0x0000000731c00000| Complete +| 399|0x0000000731e00000, 0x0000000732000000, 0x0000000732000000|100%| S|CS|TAMS 0x0000000731e00000, 0x0000000731e00000| Complete +| 400|0x0000000732000000, 0x0000000732200000, 0x0000000732200000|100%| S|CS|TAMS 0x0000000732000000, 0x0000000732000000| Complete +| 401|0x0000000732200000, 0x0000000732400000, 0x0000000732400000|100%| S|CS|TAMS 0x0000000732200000, 0x0000000732200000| Complete +| 402|0x0000000732400000, 0x0000000732400000, 0x0000000732600000| 0%| F| |TAMS 0x0000000732400000, 0x0000000732400000| Untracked +| 403|0x0000000732600000, 0x0000000732600000, 0x0000000732800000| 0%| F| |TAMS 0x0000000732600000, 0x0000000732600000| Untracked +| 404|0x0000000732800000, 0x0000000732800000, 0x0000000732a00000| 0%| F| |TAMS 0x0000000732800000, 0x0000000732800000| Untracked +| 405|0x0000000732a00000, 0x0000000732a00000, 0x0000000732c00000| 0%| F| |TAMS 0x0000000732a00000, 0x0000000732a00000| Untracked +| 406|0x0000000732c00000, 0x0000000732c00000, 0x0000000732e00000| 0%| F| |TAMS 0x0000000732c00000, 0x0000000732c00000| Untracked +| 407|0x0000000732e00000, 0x0000000732e00000, 0x0000000733000000| 0%| F| |TAMS 0x0000000732e00000, 0x0000000732e00000| Untracked +| 408|0x0000000733000000, 0x000000073308e4c0, 0x0000000733200000| 27%| E| |TAMS 0x0000000733000000, 0x0000000733000000| Complete +| 409|0x0000000733200000, 0x0000000733400000, 0x0000000733400000|100%| E|CS|TAMS 0x0000000733200000, 0x0000000733200000| Complete +| 410|0x0000000733400000, 0x0000000733600000, 0x0000000733600000|100%| E|CS|TAMS 0x0000000733400000, 0x0000000733400000| Complete +| 411|0x0000000733600000, 0x0000000733800000, 0x0000000733800000|100%| E|CS|TAMS 0x0000000733600000, 0x0000000733600000| Complete +| 412|0x0000000733800000, 0x0000000733a00000, 0x0000000733a00000|100%| E|CS|TAMS 0x0000000733800000, 0x0000000733800000| Complete +| 413|0x0000000733a00000, 0x0000000733c00000, 0x0000000733c00000|100%| E|CS|TAMS 0x0000000733a00000, 0x0000000733a00000| Complete +| 414|0x0000000733c00000, 0x0000000733e00000, 0x0000000733e00000|100%| E|CS|TAMS 0x0000000733c00000, 0x0000000733c00000| Complete +| 415|0x0000000733e00000, 0x0000000734000000, 0x0000000734000000|100%| E| |TAMS 0x0000000733e00000, 0x0000000733e00000| Complete +| 416|0x0000000734000000, 0x0000000734200000, 0x0000000734200000|100%| E|CS|TAMS 0x0000000734000000, 0x0000000734000000| Complete +| 417|0x0000000734200000, 0x0000000734400000, 0x0000000734400000|100%| E|CS|TAMS 0x0000000734200000, 0x0000000734200000| Complete +| 418|0x0000000734400000, 0x0000000734600000, 0x0000000734600000|100%| E|CS|TAMS 0x0000000734400000, 0x0000000734400000| Complete +| 419|0x0000000734600000, 0x0000000734800000, 0x0000000734800000|100%| E|CS|TAMS 0x0000000734600000, 0x0000000734600000| Complete +| 420|0x0000000734800000, 0x0000000734a00000, 0x0000000734a00000|100%| E|CS|TAMS 0x0000000734800000, 0x0000000734800000| Complete +| 421|0x0000000734a00000, 0x0000000734c00000, 0x0000000734c00000|100%| E|CS|TAMS 0x0000000734a00000, 0x0000000734a00000| Complete +| 422|0x0000000734c00000, 0x0000000734e00000, 0x0000000734e00000|100%| E|CS|TAMS 0x0000000734c00000, 0x0000000734c00000| Complete +| 423|0x0000000734e00000, 0x0000000735000000, 0x0000000735000000|100%| E|CS|TAMS 0x0000000734e00000, 0x0000000734e00000| Complete +| 424|0x0000000735000000, 0x0000000735200000, 0x0000000735200000|100%| E|CS|TAMS 0x0000000735000000, 0x0000000735000000| Complete +| 425|0x0000000735200000, 0x0000000735400000, 0x0000000735400000|100%| E|CS|TAMS 0x0000000735200000, 0x0000000735200000| Complete +| 426|0x0000000735400000, 0x0000000735600000, 0x0000000735600000|100%| E|CS|TAMS 0x0000000735400000, 0x0000000735400000| Complete +| 427|0x0000000735600000, 0x0000000735800000, 0x0000000735800000|100%| E|CS|TAMS 0x0000000735600000, 0x0000000735600000| Complete +| 428|0x0000000735800000, 0x0000000735a00000, 0x0000000735a00000|100%| E|CS|TAMS 0x0000000735800000, 0x0000000735800000| Complete +| 429|0x0000000735a00000, 0x0000000735c00000, 0x0000000735c00000|100%| E|CS|TAMS 0x0000000735a00000, 0x0000000735a00000| Complete +| 430|0x0000000735c00000, 0x0000000735e00000, 0x0000000735e00000|100%| E|CS|TAMS 0x0000000735c00000, 0x0000000735c00000| Complete +| 431|0x0000000735e00000, 0x0000000736000000, 0x0000000736000000|100%| E|CS|TAMS 0x0000000735e00000, 0x0000000735e00000| Complete +| 432|0x0000000736000000, 0x0000000736200000, 0x0000000736200000|100%| E|CS|TAMS 0x0000000736000000, 0x0000000736000000| Complete +| 433|0x0000000736200000, 0x0000000736400000, 0x0000000736400000|100%| E|CS|TAMS 0x0000000736200000, 0x0000000736200000| Complete +| 434|0x0000000736400000, 0x0000000736600000, 0x0000000736600000|100%| E|CS|TAMS 0x0000000736400000, 0x0000000736400000| Complete +| 435|0x0000000736600000, 0x0000000736800000, 0x0000000736800000|100%| E|CS|TAMS 0x0000000736600000, 0x0000000736600000| Complete +| 436|0x0000000736800000, 0x0000000736a00000, 0x0000000736a00000|100%| E|CS|TAMS 0x0000000736800000, 0x0000000736800000| Complete +| 437|0x0000000736a00000, 0x0000000736c00000, 0x0000000736c00000|100%| E|CS|TAMS 0x0000000736a00000, 0x0000000736a00000| Complete +| 438|0x0000000736c00000, 0x0000000736e00000, 0x0000000736e00000|100%| E|CS|TAMS 0x0000000736c00000, 0x0000000736c00000| Complete +| 439|0x0000000736e00000, 0x0000000737000000, 0x0000000737000000|100%| E|CS|TAMS 0x0000000736e00000, 0x0000000736e00000| Complete +| 440|0x0000000737000000, 0x0000000737200000, 0x0000000737200000|100%| E|CS|TAMS 0x0000000737000000, 0x0000000737000000| Complete +| 441|0x0000000737200000, 0x0000000737400000, 0x0000000737400000|100%| E|CS|TAMS 0x0000000737200000, 0x0000000737200000| Complete +| 442|0x0000000737400000, 0x0000000737600000, 0x0000000737600000|100%| E|CS|TAMS 0x0000000737400000, 0x0000000737400000| Complete +| 443|0x0000000737600000, 0x0000000737800000, 0x0000000737800000|100%| E|CS|TAMS 0x0000000737600000, 0x0000000737600000| Complete +| 444|0x0000000737800000, 0x0000000737a00000, 0x0000000737a00000|100%| E|CS|TAMS 0x0000000737800000, 0x0000000737800000| Complete +| 445|0x0000000737a00000, 0x0000000737c00000, 0x0000000737c00000|100%| E|CS|TAMS 0x0000000737a00000, 0x0000000737a00000| Complete +| 446|0x0000000737c00000, 0x0000000737e00000, 0x0000000737e00000|100%| E|CS|TAMS 0x0000000737c00000, 0x0000000737c00000| Complete +| 447|0x0000000737e00000, 0x0000000738000000, 0x0000000738000000|100%| E|CS|TAMS 0x0000000737e00000, 0x0000000737e00000| Complete +| 448|0x0000000738000000, 0x0000000738200000, 0x0000000738200000|100%| E|CS|TAMS 0x0000000738000000, 0x0000000738000000| Complete +| 449|0x0000000738200000, 0x0000000738400000, 0x0000000738400000|100%| E|CS|TAMS 0x0000000738200000, 0x0000000738200000| Complete +| 450|0x0000000738400000, 0x0000000738600000, 0x0000000738600000|100%| E|CS|TAMS 0x0000000738400000, 0x0000000738400000| Complete +| 451|0x0000000738600000, 0x0000000738800000, 0x0000000738800000|100%| E|CS|TAMS 0x0000000738600000, 0x0000000738600000| Complete +| 452|0x0000000738800000, 0x0000000738a00000, 0x0000000738a00000|100%| E|CS|TAMS 0x0000000738800000, 0x0000000738800000| Complete +| 453|0x0000000738a00000, 0x0000000738c00000, 0x0000000738c00000|100%| E|CS|TAMS 0x0000000738a00000, 0x0000000738a00000| Complete +| 454|0x0000000738c00000, 0x0000000738e00000, 0x0000000738e00000|100%| E|CS|TAMS 0x0000000738c00000, 0x0000000738c00000| Complete +| 455|0x0000000738e00000, 0x0000000739000000, 0x0000000739000000|100%| E|CS|TAMS 0x0000000738e00000, 0x0000000738e00000| Complete +| 456|0x0000000739000000, 0x0000000739200000, 0x0000000739200000|100%| E|CS|TAMS 0x0000000739000000, 0x0000000739000000| Complete +| 457|0x0000000739200000, 0x0000000739400000, 0x0000000739400000|100%| E|CS|TAMS 0x0000000739200000, 0x0000000739200000| Complete +| 458|0x0000000739400000, 0x0000000739600000, 0x0000000739600000|100%| E|CS|TAMS 0x0000000739400000, 0x0000000739400000| Complete +| 459|0x0000000739600000, 0x0000000739800000, 0x0000000739800000|100%| E|CS|TAMS 0x0000000739600000, 0x0000000739600000| Complete +| 460|0x0000000739800000, 0x0000000739a00000, 0x0000000739a00000|100%| E|CS|TAMS 0x0000000739800000, 0x0000000739800000| Complete +| 461|0x0000000739a00000, 0x0000000739c00000, 0x0000000739c00000|100%| E|CS|TAMS 0x0000000739a00000, 0x0000000739a00000| Complete +| 462|0x0000000739c00000, 0x0000000739e00000, 0x0000000739e00000|100%| E|CS|TAMS 0x0000000739c00000, 0x0000000739c00000| Complete +| 767|0x000000075fe00000, 0x0000000760000000, 0x0000000760000000|100%| E|CS|TAMS 0x000000075fe00000, 0x000000075fe00000| Complete +|2046|0x00000007ffc00000, 0x00000007ffd74000, 0x00000007ffe00000| 72%|OA| |TAMS 0x00000007ffd74000, 0x00000007ffd74000| Untracked +|2047|0x00000007ffe00000, 0x00000007ffe74000, 0x0000000800000000| 22%|CA| |TAMS 0x00000007ffe74000, 0x00000007ffe74000| Untracked + +Card table byte_map: [0x0000000107fd4000,0x00000001087d4000] _byte_map_base: 0x00000001047d4000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x00000001033fb6c0, (CMBitMap*) 0x00000001033fb680 + Prev Bits: [0x0000000124b30000, 0x0000000128b30000) + Next Bits: [0x0000000120b30000, 0x0000000124b30000) + +Polling page: 0x0000000102b68000 + +Metaspace: + +Usage: + Non-class: 147.68 MB used. + Class: 18.98 MB used. + Both: 166.66 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 148.44 MB ( 77%) committed, 3 nodes. + Class space: 1.00 GB reserved, 19.62 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 168.06 MB ( 14%) committed. + +Chunk freelists: + Non-Class: 10.80 MB + Class: 12.42 MB + Both: 23.22 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 241.19 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 12. +num_arena_births: 2812. +num_arena_deaths: 434. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2686. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 620. +num_chunks_taken_from_freelist: 10025. +num_chunk_merges: 224. +num_chunk_splits: 6875. +num_chunks_enlarged: 4900. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=21056Kb max_used=21056Kb free=98111Kb + bounds [0x0000000118ed0000, 0x000000011a370000, 0x0000000120330000] +CodeHeap 'profiled nmethods': size=119152Kb used=55552Kb max_used=55552Kb free=63599Kb + bounds [0x0000000111330000, 0x0000000114980000, 0x000000011878c000] +CodeHeap 'non-nmethods': size=7440Kb used=3363Kb max_used=3558Kb free=4077Kb + bounds [0x000000011878c000, 0x0000000118b1c000, 0x0000000118ed0000] + total_blobs=28079 nmethods=26975 adapters=1014 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 44.759 Thread 0x0000000c688a7600 38501 4 org.apache.spark.serializer.JavaDeserializationStream$$anon$1:: (34 bytes) +Event: 44.762 Thread 0x0000000c654af600 38502 3 org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec::productIterator (18 bytes) +Event: 44.762 Thread 0x0000000c654af600 nmethod 38502 0x0000000114972f90 code [0x0000000114973140, 0x0000000114973338] +Event: 44.765 Thread 0x0000000c688a6400 38503 3 com.google.protobuf.TextFormatEscaper$1:: (18 bytes) +Event: 44.765 Thread 0x0000000c654aea00 38504 3 com.google.protobuf.TextFormatEscaper::escapeBytes (441 bytes) +Event: 44.765 Thread 0x0000000c688a6400 nmethod 38503 0x0000000114973490 code [0x0000000114973640, 0x0000000114973888] +Event: 44.765 Thread 0x0000000c654aea00 nmethod 38504 0x0000000114973a10 code [0x0000000114973f80, 0x00000001149758f8] +Event: 44.766 Thread 0x0000000c654af000 38505 3 org.apache.spark.sql.catalyst.trees.SQLQueryContext::isValid (176 bytes) +Event: 44.766 Thread 0x0000000c654af600 38506 3 org.apache.comet.serde.PartitioningOuterClass$Partitioning$Builder:: (21 bytes) +Event: 44.766 Thread 0x0000000c654aea00 38507 3 org.apache.comet.serde.OperatorOuterClass$ShuffleWriter$Builder::buildPartial (57 bytes) +Event: 44.766 Thread 0x0000000c688a6400 38508 3 scala.collection.immutable.MapLike$$anon$1::seq (13 bytes) +Event: 44.766 Thread 0x0000000c654af600 nmethod 38506 0x0000000114976390 code [0x0000000114976540, 0x0000000114976708] +Event: 44.766 Thread 0x0000000c654af600 38509 3 org.apache.spark.sql.comet.execution.shuffle.CometShuffleWriterInputIterator::nativeIterator (14 bytes) +Event: 44.766 Thread 0x0000000c688a6400 nmethod 38508 0x0000000114976810 code [0x00000001149769c0, 0x0000000114976bb8] +Event: 44.766 Thread 0x0000000c688a6400 38510 3 org.apache.comet.CometExecIterator::enableStashMode (16 bytes) +Event: 44.766 Thread 0x0000000c654aea00 nmethod 38507 0x0000000114976d10 code [0x0000000114976f00, 0x0000000114977328] +Event: 44.766 Thread 0x0000000c654af600 nmethod 38509 0x0000000114977490 code [0x0000000114977640, 0x00000001149777c8] +Event: 44.766 Thread 0x0000000c688a6400 nmethod 38510 0x0000000114977910 code [0x0000000114977ac0, 0x0000000114977cb8] +Event: 44.766 Thread 0x0000000c654af000 nmethod 38505 0x0000000114977e10 code [0x0000000114978240, 0x0000000114979588] +Event: 44.767 Thread 0x0000000c688a7600 nmethod 38501 0x000000011a35ba90 code [0x000000011a35be00, 0x000000011a35d1e0] + +GC Heap History (20 events): +Event: 43.099 GC heap before +{Heap before GC invocations=54 (full 0): + garbage-first heap total 954368K, used 427333K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 60 young (122880K), 5 survivors (10240K) + Metaspace used 168094K, committed 169472K, reserved 1245184K + class space used 19354K, committed 19968K, reserved 1048576K +} +Event: 43.102 GC heap after +{Heap after GC invocations=55 (full 0): + garbage-first heap total 954368K, used 136887K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 168094K, committed 169472K, reserved 1245184K + class space used 19354K, committed 19968K, reserved 1048576K +} +Event: 43.273 GC heap before +{Heap before GC invocations=55 (full 0): + garbage-first heap total 954368K, used 435895K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 65 young (133120K), 5 survivors (10240K) + Metaspace used 168345K, committed 169728K, reserved 1245184K + class space used 19362K, committed 19968K, reserved 1048576K +} +Event: 43.276 GC heap after +{Heap after GC invocations=56 (full 0): + garbage-first heap total 954368K, used 137390K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 168345K, committed 169728K, reserved 1245184K + class space used 19362K, committed 19968K, reserved 1048576K +} +Event: 43.452 GC heap before +{Heap before GC invocations=56 (full 0): + garbage-first heap total 954368K, used 438446K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 65 young (133120K), 5 survivors (10240K) + Metaspace used 168649K, committed 170048K, reserved 1245184K + class space used 19365K, committed 20032K, reserved 1048576K +} +Event: 43.455 GC heap after +{Heap after GC invocations=57 (full 0): + garbage-first heap total 954368K, used 138096K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 5 young (10240K), 5 survivors (10240K) + Metaspace used 168649K, committed 170048K, reserved 1245184K + class space used 19365K, committed 20032K, reserved 1048576K +} +Event: 43.648 GC heap before +{Heap before GC invocations=57 (full 0): + garbage-first heap total 954368K, used 424816K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 60 young (122880K), 5 survivors (10240K) + Metaspace used 169313K, committed 170688K, reserved 1245184K + class space used 19405K, committed 20032K, reserved 1048576K +} +Event: 43.650 GC heap after +{Heap after GC invocations=58 (full 0): + garbage-first heap total 954368K, used 138744K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 169313K, committed 170688K, reserved 1245184K + class space used 19405K, committed 20032K, reserved 1048576K +} +Event: 43.793 GC heap before +{Heap before GC invocations=58 (full 0): + garbage-first heap total 954368K, used 417272K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 54 young (110592K), 4 survivors (8192K) + Metaspace used 169487K, committed 170816K, reserved 1245184K + class space used 19406K, committed 20032K, reserved 1048576K +} +Event: 43.795 GC heap after +{Heap after GC invocations=59 (full 0): + garbage-first heap total 954368K, used 138412K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 169487K, committed 170816K, reserved 1245184K + class space used 19406K, committed 20032K, reserved 1048576K +} +Event: 44.025 GC heap before +{Heap before GC invocations=59 (full 0): + garbage-first heap total 954368K, used 449708K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 72 young (147456K), 4 survivors (8192K) + Metaspace used 169922K, committed 171328K, reserved 1245184K + class space used 19436K, committed 20096K, reserved 1048576K +} +Event: 44.027 GC heap after +{Heap after GC invocations=60 (full 0): + garbage-first heap total 954368K, used 138976K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 169922K, committed 171328K, reserved 1245184K + class space used 19436K, committed 20096K, reserved 1048576K +} +Event: 44.170 GC heap before +{Heap before GC invocations=60 (full 0): + garbage-first heap total 954368K, used 427744K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 60 young (122880K), 4 survivors (8192K) + Metaspace used 170050K, committed 171456K, reserved 1245184K + class space used 19436K, committed 20096K, reserved 1048576K +} +Event: 44.172 GC heap after +{Heap after GC invocations=61 (full 0): + garbage-first heap total 954368K, used 139147K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 170050K, committed 171456K, reserved 1245184K + class space used 19436K, committed 20096K, reserved 1048576K +} +Event: 44.310 GC heap before +{Heap before GC invocations=61 (full 0): + garbage-first heap total 954368K, used 423819K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 57 young (116736K), 4 survivors (8192K) + Metaspace used 170204K, committed 171648K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} +Event: 44.312 GC heap after +{Heap after GC invocations=62 (full 0): + garbage-first heap total 954368K, used 139345K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 170204K, committed 171648K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} +Event: 44.459 GC heap before +{Heap before GC invocations=62 (full 0): + garbage-first heap total 954368K, used 417873K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 56 young (114688K), 4 survivors (8192K) + Metaspace used 170279K, committed 171712K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} +Event: 44.461 GC heap after +{Heap after GC invocations=63 (full 0): + garbage-first heap total 954368K, used 138883K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 170279K, committed 171712K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} +Event: 44.618 GC heap before +{Heap before GC invocations=63 (full 0): + garbage-first heap total 954368K, used 433795K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 62 young (126976K), 4 survivors (8192K) + Metaspace used 170506K, committed 171904K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} +Event: 44.620 GC heap after +{Heap after GC invocations=64 (full 0): + garbage-first heap total 954368K, used 139704K [0x0000000700000000, 0x0000000800000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 170506K, committed 171904K, reserved 1245184K + class space used 19439K, committed 20096K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.379 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.379 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.344 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-1214774261196833962.dylib +Event: 5.837 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 5.837 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 5.837 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 5.837 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 5.837 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 5.837 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 5.837 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.837 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 5.837 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 5.837 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.837 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 5.837 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 5.837 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 5.837 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 5.981 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-3296482769080348166.dylib +Event: 6.743 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-9c7a5abf-870c-4741-a8f7-18e9355385fb-libsnappyjava.dylib +Event: 39.824 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/jnilib-13936019976582104019.tmp + +Deoptimization events (20 events): +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 +Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 +Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 +Event: 44.482 Thread 0x0000000c67a9aa00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00000001195069b8 relative=0x0000000000001138 +Event: 44.482 Thread 0x0000000c67a9aa00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00000001195069b8 method=java.util.concurrent.ConcurrentHashMap.putVal(Ljava/lang/Object;Ljava/lang/Object;Z)Ljava/lang/Object; @ 162 c2 +Event: 44.482 Thread 0x0000000c67a9aa00 DEOPT PACKING pc=0x00000001195069b8 sp=0x000000017e66db70 +Event: 44.482 Thread 0x0000000c67a9aa00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017e66daa0 mode 2 +Event: 44.486 Thread 0x0000000c67a9aa00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x000000011943b6c8 relative=0x0000000000000388 +Event: 44.486 Thread 0x0000000c67a9aa00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x000000011943b6c8 method=java.util.concurrent.ConcurrentHashMap.replaceNode(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object; @ 150 c2 +Event: 44.486 Thread 0x0000000c67a9aa00 DEOPT PACKING pc=0x000000011943b6c8 sp=0x000000017e66dca0 +Event: 44.486 Thread 0x0000000c67a9aa00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017e66dbe0 mode 2 + +Classes loaded (20 events): +Event: 42.562 Loading class jdk/internal/reflect/UnsafeQualifiedStaticIntegerFieldAccessorImpl +Event: 42.562 Loading class jdk/internal/reflect/UnsafeQualifiedStaticIntegerFieldAccessorImpl done +Event: 42.724 Loading class java/nio/channels/Channels$WritableByteChannelImpl +Event: 42.724 Loading class java/nio/channels/Channels$WritableByteChannelImpl done +Event: 42.889 Loading class java/lang/org +Event: 42.889 Loading class java/lang/org done +Event: 43.056 Loading class java/lang/org +Event: 43.056 Loading class java/lang/org done +Event: 43.217 Loading class java/lang/org +Event: 43.217 Loading class java/lang/org done +Event: 43.377 Loading class java/lang/org +Event: 43.377 Loading class java/lang/org done +Event: 43.937 Loading class java/util/stream/StreamSpliterators$WrappingSpliterator +Event: 43.937 Loading class java/util/stream/StreamSpliterators$AbstractWrappingSpliterator +Event: 43.937 Loading class java/util/stream/StreamSpliterators$AbstractWrappingSpliterator done +Event: 43.937 Loading class java/util/stream/StreamSpliterators$WrappingSpliterator done +Event: 43.937 Loading class java/util/stream/StreamSpliterators +Event: 43.938 Loading class java/util/stream/StreamSpliterators done +Event: 43.938 Loading class java/util/function/BooleanSupplier +Event: 43.938 Loading class java/util/function/BooleanSupplier done + +Classes unloaded (20 events): +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1b000 'java/lang/invoke/LambdaForm$MH+0x000000c801e1b000' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1ac00 'java/lang/invoke/LambdaForm$MH+0x000000c801e1ac00' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1a800 'java/lang/invoke/LambdaForm$MH+0x000000c801e1a800' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1a400 'java/lang/invoke/LambdaForm$MH+0x000000c801e1a400' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19800 'java/lang/invoke/LambdaForm$MH+0x000000c801e19800' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19400 'java/lang/invoke/LambdaForm$MH+0x000000c801e19400' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19000 'java/lang/invoke/LambdaForm$MH+0x000000c801e19000' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e18c00' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18400 'java/lang/invoke/LambdaForm$MH+0x000000c801e18400' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18000 'java/lang/invoke/LambdaForm$MH+0x000000c801e18000' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e16c00' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16800 'java/lang/invoke/LambdaForm$MH+0x000000c801e16800' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16000 'java/lang/invoke/LambdaForm$MH+0x000000c801e16000' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e15c00' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15800 'java/lang/invoke/LambdaForm$MH+0x000000c801e15800' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15400 'java/lang/invoke/LambdaForm$MH+0x000000c801e15400' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e14c00' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14400 'java/lang/invoke/LambdaForm$MH+0x000000c801e14400' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14000 'java/lang/invoke/LambdaForm$MH+0x000000c801e14000' +Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14800 'java/lang/invoke/LambdaForm$MH+0x000000c801e14800' + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 40.510 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736dc4b30) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a34d90) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a444e8) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a47f10) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a56cc0) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368cc910) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368e3068) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368f62e0) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736909fb8) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.517 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736935678) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.547 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 +Event: 40.583 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 +Event: 40.584 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 +Event: 40.584 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 +Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b4bc58) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b55800) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b86b70) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 40.782 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737baa740) +thrown [src/hotspot/share/prims/jni.cpp, line 1112] +Event: 42.152 Thread 0x0000000c64289e00 Implicit null exception at 0x0000000119c003d4 to 0x0000000119c00cb4 +Event: 43.939 Thread 0x00000001033f4320 Implicit null exception at 0x0000000118fb6e18 to 0x0000000118fb6f50 + +VM Operations (20 events): +Event: 43.793 Executing VM operation: G1TryInitiateConcMark +Event: 43.795 Executing VM operation: G1TryInitiateConcMark done +Event: 43.951 Executing VM operation: ICBufferFull +Event: 43.951 Executing VM operation: ICBufferFull done +Event: 44.007 Executing VM operation: ICBufferFull +Event: 44.007 Executing VM operation: ICBufferFull done +Event: 44.025 Executing VM operation: G1TryInitiateConcMark +Event: 44.027 Executing VM operation: G1TryInitiateConcMark done +Event: 44.170 Executing VM operation: G1TryInitiateConcMark +Event: 44.172 Executing VM operation: G1TryInitiateConcMark done +Event: 44.310 Executing VM operation: G1TryInitiateConcMark +Event: 44.312 Executing VM operation: G1TryInitiateConcMark done +Event: 44.312 Executing VM operation: G1TryInitiateConcMark +Event: 44.312 Executing VM operation: G1TryInitiateConcMark done +Event: 44.459 Executing VM operation: G1TryInitiateConcMark +Event: 44.461 Executing VM operation: G1TryInitiateConcMark done +Event: 44.618 Executing VM operation: G1TryInitiateConcMark +Event: 44.620 Executing VM operation: G1TryInitiateConcMark done +Event: 44.620 Executing VM operation: G1TryInitiateConcMark +Event: 44.620 Executing VM operation: G1TryInitiateConcMark done + +Memory protections (20 events): +Event: 44.408 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 +Event: 44.408 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 +Event: 44.409 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 +Event: 44.409 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 +Event: 44.515 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 +Event: 44.515 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 +Event: 44.515 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 +Event: 44.515 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 +Event: 44.549 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 +Event: 44.549 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 +Event: 44.549 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 +Event: 44.549 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 +Event: 44.655 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 +Event: 44.655 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 +Event: 44.655 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 +Event: 44.655 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 +Event: 44.687 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 +Event: 44.688 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 +Event: 44.688 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 +Event: 44.688 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 + +Nmethod flushes (20 events): +Event: 43.577 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001129c0490 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bd9110 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bdbb90 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bebf90 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bec490 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bec990 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bf2c90 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112cab890 +Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112cf9e90 +Event: 43.579 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112ef4b90 +Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bb810 +Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bbf10 +Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bcd90 +Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131c6690 +Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131c7510 +Event: 43.582 Thread 0x0000000c688a6a00 flushing osr nmethod 0x0000000113970910 +Event: 43.582 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000113bcc690 +Event: 43.582 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000113bccb10 +Event: 43.582 Thread 0x0000000c688a6a00 flushing osr nmethod 0x0000000113bdd910 +Event: 43.585 Thread 0x0000000c688a6a00 flushing nmethod 0x000000011414f590 + +Events (20 events): +Event: 44.408 Thread 0x0000000c67a99e00 Thread added: 0x0000000c638b1800 +Event: 44.408 Thread 0x0000000c638b1800 Thread added: 0x0000000c638b1e00 +Event: 44.409 Thread 0x0000000c638b1e00 Thread exited: 0x0000000c638b1e00 +Event: 44.409 Thread 0x0000000c638b1800 Thread exited: 0x0000000c638b1800 +Event: 44.515 Thread 0x00000001033f4320 Thread added: 0x0000000c638b3600 +Event: 44.515 Thread 0x0000000c638b3600 Thread added: 0x0000000c638d0000 +Event: 44.515 Thread 0x0000000c638d0000 Thread exited: 0x0000000c638d0000 +Event: 44.516 Thread 0x0000000c638b3600 Thread exited: 0x0000000c638b3600 +Event: 44.549 Thread 0x0000000c67a9a400 Thread added: 0x0000000c638b3600 +Event: 44.549 Thread 0x0000000c638b3600 Thread added: 0x0000000c638d0000 +Event: 44.549 Thread 0x0000000c638d0000 Thread exited: 0x0000000c638d0000 +Event: 44.549 Thread 0x0000000c638b3600 Thread exited: 0x0000000c638b3600 +Event: 44.655 Thread 0x00000001033f4320 Thread added: 0x0000000c638d0c00 +Event: 44.655 Thread 0x0000000c638d0c00 Thread added: 0x0000000c638d1200 +Event: 44.655 Thread 0x0000000c638d1200 Thread exited: 0x0000000c638d1200 +Event: 44.655 Thread 0x0000000c638d0c00 Thread exited: 0x0000000c638d0c00 +Event: 44.687 Thread 0x0000000c67a9a400 Thread added: 0x0000000c638d0c00 +Event: 44.688 Thread 0x0000000c638d0c00 Thread added: 0x0000000c638d1200 +Event: 44.688 Thread 0x0000000c638d1200 Thread exited: 0x0000000c638d1200 +Event: 44.688 Thread 0x0000000c638d0c00 Thread exited: 0x0000000c638d0c00 + + +Dynamic libraries: +0x0000000102b80000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x00000001087ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x0000000102ba8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x0000000102bd8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x0000000102c28000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x0000000102e9c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x0000000102ec4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x0000000102ee4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x0000000102f08000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x0000000102e88000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x0000000102f24000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x0000000102f38000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x000000012acd8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000151e04000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-1214774261196833962.dylib +0x000000012ad20000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-3296482769080348166.dylib +0x000000012ad4c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-9c7a5abf-870c-4741-a8f7-18e9355385fb-libsnappyjava.dylib +0x0000000102e74000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/jnilib-13936019976582104019.tmp + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:41 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 6.34 7.11 5.42 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(3840832k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/hs_err_pid59323.log b/spark/hs_err_pid59323.log new file mode 100644 index 0000000000..9411333aa6 --- /dev/null +++ b/spark/hs_err_pid59323.log @@ -0,0 +1,1626 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x0000000108449f64, pid=59323, tid=64019 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 08:59:57 2026 MDT elapsed time: 21.549643 seconds (0d 0h 0m 21s) + +--------------- T H R E A D --------------- + +Current thread (0x000000093d9d8600): JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x000000017e688000,0x000000017ea8b000)] + +Stack: [0x000000017e688000,0x000000017ea8b000], sp=0x000000017ea72120, free space=4008k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c +C [libcomet-6348563280641709946.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 +C [libcomet-6348563280641709946.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 +C [libcomet-6348563280641709946.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c +C [libcomet-6348563280641709946.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 +C [libcomet-6348563280641709946.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 +C [libcomet-6348563280641709946.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 +C [libcomet-6348563280641709946.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 +C [libcomet-6348563280641709946.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-6348563280641709946.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-6348563280641709946.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-6348563280641709946.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-6348563280641709946.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-6348563280641709946.dylib+0x106628] __rust_try+0x20 +C [libcomet-6348563280641709946.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-6348563280641709946.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-6348563280641709946.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-6348563280641709946.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3997+0x000000e0021c93d0.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3988+0x000000e0021c8000.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2525+0x000000e001df5ed0.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3997+0x000000e0021c93d0.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3988+0x000000e0021c8000.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2525+0x000000e001df5ed0.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0xba9cde559ff17bfe + +Registers: + x0=0x000000e0021b54c0 x1=0x0000000000000010 x2=0x000000094432ea20 x3=0x000000017ea77298 + x4=0x0000000000000238 x5=0x0000000000000018 x6=0x000000094b9050e0 x7=0xfffff0003ffff800 + x8=0x00000000021b54c0 x9=0x0000000000000000 x10=0x000000e000000000 x11=0x000000093f17d3c0 +x12=0x0000000942256490 x13=0x0000000000000001 x14=0x00000001027d5a08 x15=0x000000093f17c000 +x16=0x000000018c8009d0 x17=0x000000093d839400 x18=0x0000000000000000 x19=0x000000093d9d8600 +x20=0x0000000000000001 x21=0x0000000102bae3a0 x22=0xba9cde559ff17bea x23=0x0000000108ba9088 +x24=0x0000000000000000 x25=0x0000000108badf38 x26=0x0000000136fd7a48 x27=0x00000004d7d9d260 +x28=0x000000093d9d8600 fp=0x000000017ea72160 lr=0x0000000108449f38 sp=0x000000017ea72120 +pc=0x0000000108449f64 cpsr=0x0000000040001000 + +Register to memory mapping: + + x0=0x000000e0021b54c0 is a pointer to class: +org.apache.comet.CometHandleBatchIterator {0x000000e0021b54c0} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x00000001370504c8) + - method ordering: Array(0x000000e000913f08) + - local interfaces: Array(0x000000e0005d4fb8) + - trans. interfaces: Array(0x000000e0005d4fb8) + - constants: constant pool [62]/operands[4] {0x000000013704bd98} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000137050708 + - class loader data: loader data: 0x000000093f007d40 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x000000e0005d4f48) + - nest members: Array(0x000000e0005d4f48) + - permitted subclasses: Array(0x000000e0005d4f48) + - java mirror: a 'java/lang/Class'{0x00000004e01635c0} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x000000e0021b5690) + - itable length 2 (start addr: 0x000000e0021b56c0) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x1=0x0000000000000010 is an unknown value + x2=0x000000094432ea20 points into unknown readable memory: 0x00000004ff8bdfa8 | a8 df 8b ff 04 00 00 00 + x3=0x000000017ea77298 is pointing into the stack for thread: 0x000000093d9d8600 + x4=0x0000000000000238 is an unknown value + x5=0x0000000000000018 is an unknown value + x6=0x000000094b9050e0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=35345600 is a compressed pointer to class: 0x000000e0021b54c0 +org.apache.comet.CometHandleBatchIterator {0x000000e0021b54c0} + - instance size: 2 + - klass size: 67 + - access: public synchronized + - state: fully_initialized + - name: 'org/apache/comet/CometHandleBatchIterator' + - super: 'java/lang/Object' + - sub: + - arrays: NULL + - methods: Array(0x00000001370504c8) + - method ordering: Array(0x000000e000913f08) + - local interfaces: Array(0x000000e0005d4fb8) + - trans. interfaces: Array(0x000000e0005d4fb8) + - constants: constant pool [62]/operands[4] {0x000000013704bd98} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000137050708 + - class loader data: loader data: 0x000000093f007d40 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} + - source file: 'CometHandleBatchIterator.java' + - class annotations: Array(0x0000000000000000) + - class type annotations: Array(0x0000000000000000) + - field annotations: Array(0x0000000000000000) + - field type annotations: Array(0x0000000000000000) + - inner classes: Array(0x000000e0005d4f48) + - nest members: Array(0x000000e0005d4f48) + - permitted subclasses: Array(0x000000e0005d4f48) + - java mirror: a 'java/lang/Class'{0x00000004e01635c0} = 'org/apache/comet/CometHandleBatchIterator' + - vtable length 6 (start addr: 0x000000e0021b5690) + - itable length 2 (start addr: 0x000000e0021b56c0) + - ---- static fields (0 words): + - ---- non-static fields (1 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 + - non-static oop maps: 12-12 + x9=0x0 is NULL +x10=0x000000e000000000 is pointing into metadata +x11=0x000000093f17d3c0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +x12=0x0000000942256490 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x13=0x0000000000000001 is an unknown value +x14=0x00000001027d5a08 points into unknown readable memory: 0x000fd05c80438aa8 | a8 8a 43 80 5c d0 0f 00 +x15=0x000000093f17c000 points into unknown readable memory: 0xffffffffffffffff | ff ff ff ff ff ff ff ff +x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 +x17=0x000000093d839400 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 +x18=0x0 is NULL +x19=0x000000093d9d8600 is a thread +x20=0x0000000000000001 is an unknown value +x21=0x0000000102bae3a0 is a global jni handle +x22=0xba9cde559ff17bea is an unknown value +x23=0x0000000108ba9088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 +x24=0x0 is NULL +x25=0x0000000108badf38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 +x26=0x0000000136fd7a48 is pointing into metadata +x27=0x00000004d7d9d260 is an oop: java.lang.Class +{0x00000004d7d9d260} - klass: 'java/lang/Class' + - ---- fields (total size 15 words): + - private volatile transient 'classRedefinedCount' 'I' @12 0 + - private volatile transient 'cachedConstructor' 'Ljava/lang/reflect/Constructor;' @40 NULL (0) + - private transient 'name' 'Ljava/lang/String;' @44 "scala.collection.AbstractSeq"{0x00000004d7d9d2d8} (9afb3a5b) + - private transient 'module' 'Ljava/lang/Module;' @48 a 'java/lang/Module'{0x00000004d7aedbf8} (9af5db7f) + - private final 'classLoader' 'Ljava/lang/ClassLoader;' @52 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} (9af5d6ee) + - private transient 'classData' 'Ljava/lang/Object;' @56 NULL (0) + - private transient 'packageName' 'Ljava/lang/String;' @60 "scala.collection"{0x00000004d7aec308} (9af5d861) + - private final 'componentType' 'Ljava/lang/Class;' @64 NULL (0) + - private volatile transient 'reflectionData' 'Ljava/lang/ref/SoftReference;' @68 a 'java/lang/ref/SoftReference'{0x00000004dc05e488} (9b80bc91) + - private volatile transient 'genericInfo' 'Lsun/reflect/generics/repository/ClassRepository;' @72 NULL (0) + - private volatile transient 'enumConstants' '[Ljava/lang/Object;' @76 NULL (0) + - private volatile transient 'enumConstantDirectory' 'Ljava/util/Map;' @80 NULL (0) + - private volatile transient 'annotationData' 'Ljava/lang/Class$AnnotationData;' @84 NULL (0) + - private volatile transient 'annotationType' 'Lsun/reflect/annotation/AnnotationType;' @88 NULL (0) + - transient 'classValueMap' 'Ljava/lang/ClassValue$ClassValueMap;' @92 NULL (0) + - signature: Lscala/collection/AbstractSeq; + - fake entry for mirror: 'scala/collection/AbstractSeq' + - fake entry for array: 'scala/collection/AbstractSeq'[] + - fake entry for oop_size: 15 + - fake entry for static_oop_field_count: 1 + - private static transient synthetic '$jacocoData' '[Z' @112 [Z{0x00000004d8669470} (9b0cd28e) +x28=0x000000093d9d8600 is a thread + + +Top of Stack: (sp=0x000000017ea72120) +0x000000017ea72120: 0000000136fd7a48 000000009c5e19cd +0x000000017ea72130: 000000017ea89818 00000004e2f0d270 +0x000000017ea72140: 0000000000000000 0000000108bf7f90 +0x000000017ea72150: 000000017ea89738 0000000123a6772c +0x000000017ea72160: 000000017ea725f0 000000014e74ce4c +0x000000017ea72170: 0000000000000000 000000015166e880 +0x000000017ea72180: 000000017ea72190 000000014e786d54 +0x000000017ea72190: 000000094432ea20 0000000102bae3a0 +0x000000017ea721a0: 0000000108449e08 000000093f194b90 +0x000000017ea721b0: 0000000108bb2ad0 0000000108bb2ad0 +0x000000017ea721c0: 000000093d9d88b0 000000093d9d88b0 +0x000000017ea721d0: 000000015167ae90 000000004e74ee14 +0x000000017ea721e0: 0000000000000000 000000017ea72270 +0x000000017ea721f0: 000000093d9d88b0 000000015167ae90 +0x000000017ea72200: 000000003d9d8600 000000017ea72860 +0x000000017ea72210: 000000017ea72a00 8000000000000020 +0x000000017ea72220: 000000015167ae90 000000093f194b90 +0x000000017ea72230: 000000015167ae90 8000000000000020 +0x000000017ea72240: 000000015167ae90 000000093d9d88b0 +0x000000017ea72250: 000000093d9d88b0 0000000000000002 +0x000000017ea72260: 000000093d839400 0000000000000000 +0x000000017ea72270: 000000093d9d88b0 0000000000000002 +0x000000017ea72280: 000000093d839400 8000000000000020 +0x000000017ea72290: 000000015167ae90 000000093d8394d0 +0x000000017ea722a0: 000000017ea722e0 000000014e7afc28 +0x000000017ea722b0: 0000000000000000 0000000000000000 +0x000000017ea722c0: 000000093d9d88b0 000000015166e888 +0x000000017ea722d0: 0000000000000000 000000093d9d88b0 +0x000000017ea722e0: 000000093d9d88b0 0000000000000002 +0x000000017ea722f0: 0000000000000008 000000015166e888 +0x000000017ea72300: 000000017ea72340 000000014e787d8c +0x000000017ea72310: 000000017ea723f8 000000093d9d88b0 + +Instructions: (pc=0x0000000108449f64) +0x0000000108449e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 +0x0000000108449e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 +0x0000000108449e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 +0x0000000108449e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 +0x0000000108449ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 +0x0000000108449eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 +0x0000000108449ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 +0x0000000108449ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 +0x0000000108449ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 +0x0000000108449ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a +0x0000000108449f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 +0x0000000108449f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa +0x0000000108449f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 +0x0000000108449f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 +0x0000000108449f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 +0x0000000108449f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b +0x0000000108449f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 +0x0000000108449f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 +0x0000000108449f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 +0x0000000108449f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb +0x0000000108449fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 +0x0000000108449fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 +0x0000000108449fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 +0x0000000108449fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 +0x0000000108449fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 +0x0000000108449ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 +0x000000010844a004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b +0x000000010844a014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 +0x000000010844a024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 +0x000000010844a034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 +0x000000010844a044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 +0x000000010844a054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x0000000136fd7a48 is pointing into metadata +stack at sp + 1 slots: 2623412685 is a compressed pointer to object: [B +{0x00000004e2f0ce68} - klass: {type array byte} + - length: 50 +stack at sp + 2 slots: 0x000000017ea89818 is pointing into the stack for thread: 0x000000093d9d8600 +stack at sp + 3 slots: 0x00000004e2f0d270 is an oop: scala.Tuple2 +{0x00000004e2f0d270} - klass: 'scala/Tuple2' + - ---- fields (total size 3 words): + - public final '_1' 'Ljava/lang/Object;' @12 a 'org/apache/arrow/c/ArrowArray'[1] {0x00000004e2f0cf28} (9c5e19e5) + - public final '_2' 'Ljava/lang/Object;' @16 a 'org/apache/arrow/c/ArrowSchema'[1] {0x00000004e2f0cf40} (9c5e19e8) +stack at sp + 4 slots: 0x0 is NULL +stack at sp + 5 slots: 0x0000000108bf7f90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 +stack at sp + 6 slots: 0x000000017ea89738 is pointing into the stack for thread: 0x000000093d9d8600 +stack at sp + 7 slots: 0x0000000123a6772c is at code_begin+44 in an Interpreter codelet +result handlers for native calls [0x0000000123a67700, 0x0000000123a67780] 128 bytes + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x00000009461f2900, length=73, elements={ +0x0000000102b94610, 0x000000093e870000, 0x000000093e870600, 0x000000093e871200, +0x000000093e871800, 0x000000093e871e00, 0x000000093e872400, 0x000000093e872a00, +0x000000093e873000, 0x000000093e873600, 0x000000093e8daa00, 0x000000093dddaa00, +0x000000093e6c9e00, 0x000000093e6cb000, 0x000000093e6cb600, 0x000000093e738000, +0x000000093e738600, 0x000000093e738c00, 0x000000093e73b000, 0x000000093e73b600, +0x000000093de9c000, 0x000000093de9c600, 0x000000093de9cc00, 0x000000093de9d200, +0x000000093de9d800, 0x000000093de9de00, 0x000000093e750600, 0x000000093e752400, +0x000000093e752a00, 0x000000093e532a00, 0x000000093e784000, 0x000000093e784c00, +0x000000093e785200, 0x000000093e785800, 0x000000093e785e00, 0x000000093e786a00, +0x000000093e674c00, 0x000000093e674600, 0x000000093e675200, 0x000000093e533000, +0x000000093de29800, 0x000000093dd41800, 0x000000093e7e4000, 0x000000093e7f4600, +0x000000093e7f5e00, 0x000000093e7f6a00, 0x000000093d07d200, 0x000000093df77000, +0x000000093d9bf600, 0x000000093d9d8000, 0x000000093d9d8600, 0x000000093d9d8c00, +0x000000093d9d9200, 0x000000093d9ed200, 0x000000093d9ede00, 0x000000093d9ee400, +0x000000093d9ef000, 0x000000093d156400, 0x000000093e8db600, 0x000000093d9d9e00, +0x000000093d96b600, 0x000000093db16400, 0x000000093db17000, 0x000000093db17600, +0x000000093db68000, 0x000000093db68600, 0x000000093dbba400, 0x000000093dbbaa00, +0x000000093dbbb000, 0x000000093dbbb600, 0x000000093e8d8000, 0x000000093db6b600, +0x000000093b2acc00 +} + +Java Threads: ( => current thread ) + 0x0000000102b94610 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4355, stack(0x000000016db74000,0x000000016df77000)] + 0x000000093e870000 JavaThread "Reference Handler" daemon [_thread_blocked, id=23555, stack(0x000000016ebc8000,0x000000016efcb000)] + 0x000000093e870600 JavaThread "Finalizer" daemon [_thread_blocked, id=31747, stack(0x000000016efd4000,0x000000016f3d7000)] + 0x000000093e871200 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30467, stack(0x000000016f4f8000,0x000000016f8fb000)] + 0x000000093e871800 JavaThread "Service Thread" daemon [_thread_blocked, id=29955, stack(0x000000016f904000,0x000000016fd07000)] + 0x000000093e871e00 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=26371, stack(0x000000016fd10000,0x0000000170113000)] + 0x000000093e872400 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=29699, stack(0x000000017011c000,0x000000017031f000)] + 0x000000093e872a00 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=29187, stack(0x0000000170328000,0x000000017052b000)] + 0x000000093e873000 JavaThread "Sweeper thread" daemon [_thread_blocked, id=28931, stack(0x0000000170534000,0x0000000170937000)] + 0x000000093e873600 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27395, stack(0x0000000170940000,0x0000000170d43000)] + 0x000000093e8daa00 JavaThread "Notification Thread" daemon [_thread_blocked, id=42755, stack(0x0000000171788000,0x0000000171b8b000)] + 0x000000093dddaa00 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=44035, stack(0x0000000174484000,0x0000000174887000)] + 0x000000093e6c9e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=61699, stack(0x00000001754d8000,0x00000001758db000)] + 0x000000093e6cb000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=45315, stack(0x00000001758e4000,0x0000000175ce7000)] + 0x000000093e6cb600 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=61443, stack(0x0000000175cf0000,0x00000001760f3000)] + 0x000000093e738000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=60931, stack(0x00000001760fc000,0x00000001764ff000)] + 0x000000093e738600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=46083, stack(0x0000000176508000,0x000000017690b000)] + 0x000000093e738c00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=46339, stack(0x0000000176914000,0x0000000176d17000)] + 0x000000093e73b000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=46595, stack(0x0000000176d20000,0x0000000177123000)] + 0x000000093e73b600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=46851, stack(0x000000017712c000,0x000000017752f000)] + 0x000000093de9c000 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=59651, stack(0x0000000177538000,0x000000017793b000)] + 0x000000093de9c600 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=47107, stack(0x0000000177944000,0x0000000177d47000)] + 0x000000093de9cc00 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=59139, stack(0x0000000177d50000,0x0000000178153000)] + 0x000000093de9d200 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=47875, stack(0x000000017815c000,0x000000017855f000)] + 0x000000093de9d800 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=48387, stack(0x0000000178568000,0x000000017896b000)] + 0x000000093de9de00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=48899, stack(0x0000000178974000,0x0000000178d77000)] + 0x000000093e750600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=49155, stack(0x0000000178d80000,0x0000000179183000)] + 0x000000093e752400 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58371, stack(0x000000017918c000,0x000000017958f000)] + 0x000000093e752a00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=49411, stack(0x0000000179598000,0x000000017999b000)] + 0x000000093e532a00 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=57603, stack(0x00000001799a4000,0x0000000179da7000)] + 0x000000093e784000 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=49923, stack(0x0000000179db0000,0x000000017a1b3000)] + 0x000000093e784c00 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=57091, stack(0x000000017a1bc000,0x000000017a5bf000)] + 0x000000093e785200 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=50435, stack(0x000000017a5c8000,0x000000017a9cb000)] + 0x000000093e785800 JavaThread "task-abort-timer" daemon [_thread_blocked, id=56835, stack(0x000000017a9d4000,0x000000017add7000)] + 0x000000093e785e00 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=50947, stack(0x000000017ade0000,0x000000017b1e3000)] + 0x000000093e786a00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=56067, stack(0x000000017b1ec000,0x000000017b5ef000)] + 0x000000093e674c00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=51203, stack(0x000000017b5f8000,0x000000017b9fb000)] + 0x000000093e674600 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=51459, stack(0x000000017ba04000,0x000000017be07000)] + 0x000000093e675200 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=51715, stack(0x000000017be10000,0x000000017c213000)] + 0x000000093e533000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=55043, stack(0x000000017c21c000,0x000000017c61f000)] + 0x000000093de29800 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54531, stack(0x000000017c628000,0x000000017ca2b000)] + 0x000000093dd41800 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=52227, stack(0x000000017ca34000,0x000000017ce37000)] + 0x000000093e7e4000 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=53763, stack(0x000000017ce40000,0x000000017d243000)] + 0x000000093e7f4600 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=52739, stack(0x000000017d24c000,0x000000017d64f000)] + 0x000000093e7f5e00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=52995, stack(0x000000017d658000,0x000000017da5b000)] + 0x000000093e7f6a00 JavaThread "Timer-0" [_thread_blocked, id=65539, stack(0x000000017da64000,0x000000017de67000)] + 0x000000093d07d200 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=86787, stack(0x000000017de70000,0x000000017e273000)] + 0x000000093df77000 JavaThread "process reaper" daemon [_thread_blocked, id=33551, stack(0x0000000174890000,0x00000001748c7000)] + 0x000000093d9bf600 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=43287, stack(0x0000000171370000,0x0000000171773000)] + 0x000000093d9d8000 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=33315, stack(0x000000017e27c000,0x000000017e67f000)] +=>0x000000093d9d8600 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x000000017e688000,0x000000017ea8b000)] + 0x000000093d9d8c00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=30995, stack(0x000000017ea94000,0x000000017ee97000)] + 0x000000093d9d9200 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=86275, stack(0x000000017eea0000,0x000000017f2a3000)] + 0x000000093d9ed200 JavaThread "process reaper" daemon [_thread_blocked, id=85251, stack(0x000000016f46c000,0x000000016f4a3000)] + 0x000000093d9ede00 JavaThread "process reaper" daemon [_thread_blocked, id=66051, stack(0x000000016f4ac000,0x000000016f4e3000)] + 0x000000093d9ee400 JavaThread "process reaper" daemon [_thread_blocked, id=66307, stack(0x0000000174adc000,0x0000000174b13000)] + 0x000000093d9ef000 JavaThread "process reaper" daemon [_thread_blocked, id=83203, stack(0x0000000174b1c000,0x0000000174b53000)] + 0x000000093d156400 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=84787, stack(0x000000017f2ac000,0x000000017f6af000)] + 0x000000093e8db600 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=67379, stack(0x000000017f6b8000,0x000000017fabb000)] + 0x000000093d9d9e00 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=84035, stack(0x000000017fac4000,0x000000017fec7000)] + 0x000000093d96b600 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=82483, stack(0x0000000300004000,0x0000000300407000)] + 0x000000093db16400 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=82803, stack(0x0000000300410000,0x0000000300813000)] + 0x000000093db17000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=67103, stack(0x000000030081c000,0x0000000300c1f000)] + 0x000000093db17600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=81667, stack(0x0000000300c28000,0x000000030102b000)] + 0x000000093db68000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81411, stack(0x0000000301034000,0x0000000301437000)] + 0x000000093db68600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=80899, stack(0x0000000301440000,0x0000000301843000)] + 0x000000093dbba400 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=28679, stack(0x0000000170d4c000,0x0000000170f4f000)] + 0x000000093dbbaa00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=27919, stack(0x0000000170f58000,0x000000017115b000)] + 0x000000093dbbb000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=69907, stack(0x0000000171164000,0x0000000171367000)] + 0x000000093dbbb600 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=69643, stack(0x00000001748d0000,0x0000000174ad3000)] + 0x000000093e8d8000 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=28179, stack(0x0000000302288000,0x000000030248b000)] + 0x000000093db6b600 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=85511, stack(0x00000003026a0000,0x0000000302aa3000)] + 0x000000093b2acc00 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=33039, stack(0x0000000302aac000,0x0000000302caf000)] + +Other Threads: + 0x0000000102ba64e0 VMThread "VM Thread" [stack: 0x000000016e9bc000,0x000000016ebbf000] [id=19971] + 0x000000093e3e4300 WatcherThread [stack: 0x0000000171b94000,0x0000000171d97000] [id=42243] + 0x0000000102b9e340 GCTaskThread "GC Thread#0" [stack: 0x000000016df80000,0x000000016e183000] [id=14595] + 0x000000093ddbcf00 GCTaskThread "GC Thread#1" [stack: 0x0000000171da0000,0x0000000171fa3000] [id=41731] + 0x000000093ddbd200 GCTaskThread "GC Thread#2" [stack: 0x0000000171fac000,0x00000001721af000] [id=41219] + 0x000000093ddbd500 GCTaskThread "GC Thread#3" [stack: 0x00000001721b8000,0x00000001723bb000] [id=40963] + 0x000000093ddbd800 GCTaskThread "GC Thread#4" [stack: 0x00000001723c4000,0x00000001725c7000] [id=40707] + 0x000000093ddbdb00 GCTaskThread "GC Thread#5" [stack: 0x00000001725d0000,0x00000001727d3000] [id=34563] + 0x000000093ddbde00 GCTaskThread "GC Thread#6" [stack: 0x00000001727dc000,0x00000001729df000] [id=35075] + 0x000000093ddbe100 GCTaskThread "GC Thread#7" [stack: 0x00000001729e8000,0x0000000172beb000] [id=40195] + 0x000000093ddbe400 GCTaskThread "GC Thread#8" [stack: 0x0000000172bf4000,0x0000000172df7000] [id=39683] + 0x000000093ddbe700 GCTaskThread "GC Thread#9" [stack: 0x0000000172e00000,0x0000000173003000] [id=35587] + 0x000000093ddbea00 GCTaskThread "GC Thread#10" [stack: 0x000000017300c000,0x000000017320f000] [id=36099] + 0x000000093ddbed00 GCTaskThread "GC Thread#11" [stack: 0x0000000173218000,0x000000017341b000] [id=39171] + 0x000000093ddbf000 GCTaskThread "GC Thread#12" [stack: 0x0000000173424000,0x0000000173627000] [id=38659] + 0x000000093ddbf300 GCTaskThread "GC Thread#13" [stack: 0x0000000173630000,0x0000000173833000] [id=38147] + 0x000000093ddbf600 GCTaskThread "GC Thread#14" [stack: 0x000000017383c000,0x0000000173a3f000] [id=37635] + 0x000000093ddbf900 GCTaskThread "GC Thread#15" [stack: 0x0000000173a48000,0x0000000173c4b000] [id=37123] + 0x000000093ddbfc00 GCTaskThread "GC Thread#16" [stack: 0x0000000173c54000,0x0000000173e57000] [id=36611] + 0x000000093e4e8000 GCTaskThread "GC Thread#17" [stack: 0x0000000173e60000,0x0000000174063000] [id=43523] + 0x000000093e4e8300 GCTaskThread "GC Thread#18" [stack: 0x000000017406c000,0x000000017426f000] [id=65283] + 0x000000093e4e8600 GCTaskThread "GC Thread#19" [stack: 0x0000000174278000,0x000000017447b000] [id=64771] + 0x0000000102b9f030 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016e18c000,0x000000016e38f000] [id=12803] + 0x0000000102b9f9a0 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016e398000,0x000000016e59b000] [id=13827] + 0x000000093e3e5b00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174ca8000,0x0000000174eab000] [id=63235] + 0x000000093e3e4900 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000174eb4000,0x00000001750b7000] [id=62723] + 0x000000093e3e4600 ConcurrentGCThread "G1 Conc#3" [stack: 0x00000001750c0000,0x00000001752c3000] [id=44803] + 0x000000093ddbc600 ConcurrentGCThread "G1 Conc#4" [stack: 0x00000001752cc000,0x00000001754cf000] [id=62211] + 0x0000000102ba1a60 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016e5a4000,0x000000016e7a7000] [id=16643] + 0x0000000102ba2360 ConcurrentGCThread "G1 Service" [stack: 0x000000016e7b0000,0x000000016e9b3000] [id=17155] + +Threads with active compile tasks: + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x00000004d5000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x000000e000000000-0x000000e000bc0000-0x000000e000bc0000), size 12320768, SharedBaseAddress: 0x000000e000000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x000000e001000000-0x000000e041000000, reserved size: 1073741824 +Narrow klass base: 0x000000e000000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 342016K, used 264885K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 59 young (120832K), 4 survivors (8192K) + Metaspace used 151478K, committed 152768K, reserved 1245184K + class space used 18229K, committed 18816K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x00000004d5000000, 0x00000004d5200000, 0x00000004d5200000|100%|HS| |TAMS 0x00000004d5200000, 0x00000004d5000000| Complete +| 1|0x00000004d5200000, 0x00000004d5400000, 0x00000004d5400000|100%|HS| |TAMS 0x00000004d5400000, 0x00000004d5200000| Complete +| 2|0x00000004d5400000, 0x00000004d5600000, 0x00000004d5600000|100%|HC| |TAMS 0x00000004d5600000, 0x00000004d5400000| Complete +| 3|0x00000004d5600000, 0x00000004d5800000, 0x00000004d5800000|100%|HS| |TAMS 0x00000004d5800000, 0x00000004d5600000| Complete +| 4|0x00000004d5800000, 0x00000004d5a00000, 0x00000004d5a00000|100%|HC| |TAMS 0x00000004d5a00000, 0x00000004d5800000| Complete +| 5|0x00000004d5a00000, 0x00000004d5c00000, 0x00000004d5c00000|100%|HC| |TAMS 0x00000004d5c00000, 0x00000004d5a00000| Complete +| 6|0x00000004d5c00000, 0x00000004d5e00000, 0x00000004d5e00000|100%|HC| |TAMS 0x00000004d5e00000, 0x00000004d5c00000| Complete +| 7|0x00000004d5e00000, 0x00000004d6000000, 0x00000004d6000000|100%|HC| |TAMS 0x00000004d6000000, 0x00000004d5e00000| Complete +| 8|0x00000004d6000000, 0x00000004d6200000, 0x00000004d6200000|100%|HC| |TAMS 0x00000004d6200000, 0x00000004d6000000| Complete +| 9|0x00000004d6200000, 0x00000004d6400000, 0x00000004d6400000|100%|HC| |TAMS 0x00000004d6400000, 0x00000004d6200000| Complete +| 10|0x00000004d6400000, 0x00000004d6600000, 0x00000004d6600000|100%|HC| |TAMS 0x00000004d6600000, 0x00000004d6400000| Complete +| 11|0x00000004d6600000, 0x00000004d6800000, 0x00000004d6800000|100%|HS| |TAMS 0x00000004d6800000, 0x00000004d6600000| Complete +| 12|0x00000004d6800000, 0x00000004d6a00000, 0x00000004d6a00000|100%|HS| |TAMS 0x00000004d6a00000, 0x00000004d6800000| Complete +| 13|0x00000004d6a00000, 0x00000004d6c00000, 0x00000004d6c00000|100%|HC| |TAMS 0x00000004d6c00000, 0x00000004d6a00000| Complete +| 14|0x00000004d6c00000, 0x00000004d6e00000, 0x00000004d6e00000|100%|HS| |TAMS 0x00000004d6e00000, 0x00000004d6c00000| Complete +| 15|0x00000004d6e00000, 0x00000004d7000000, 0x00000004d7000000|100%|HC| |TAMS 0x00000004d7000000, 0x00000004d6e00000| Complete +| 16|0x00000004d7000000, 0x00000004d7200000, 0x00000004d7200000|100%|HS| |TAMS 0x00000004d7200000, 0x00000004d7000000| Complete +| 17|0x00000004d7200000, 0x00000004d7400000, 0x00000004d7400000|100%|HS| |TAMS 0x00000004d7400000, 0x00000004d7200000| Complete +| 18|0x00000004d7400000, 0x00000004d7600000, 0x00000004d7600000|100%|HC| |TAMS 0x00000004d7600000, 0x00000004d7400000| Complete +| 19|0x00000004d7600000, 0x00000004d7800000, 0x00000004d7800000|100%| O| |TAMS 0x00000004d7800000, 0x00000004d7600000| Untracked +| 20|0x00000004d7800000, 0x00000004d7a00000, 0x00000004d7a00000|100%| O| |TAMS 0x00000004d7a00000, 0x00000004d7800000| Untracked +| 21|0x00000004d7a00000, 0x00000004d7c00000, 0x00000004d7c00000|100%| O| |TAMS 0x00000004d7c00000, 0x00000004d7a00000| Untracked +| 22|0x00000004d7c00000, 0x00000004d7e00000, 0x00000004d7e00000|100%| O| |TAMS 0x00000004d7e00000, 0x00000004d7c00000| Untracked +| 23|0x00000004d7e00000, 0x00000004d8000000, 0x00000004d8000000|100%| O| |TAMS 0x00000004d8000000, 0x00000004d7e00000| Untracked +| 24|0x00000004d8000000, 0x00000004d8200000, 0x00000004d8200000|100%| O| |TAMS 0x00000004d8200000, 0x00000004d8000000| Untracked +| 25|0x00000004d8200000, 0x00000004d8400000, 0x00000004d8400000|100%| O| |TAMS 0x00000004d8400000, 0x00000004d8200000| Untracked +| 26|0x00000004d8400000, 0x00000004d8600000, 0x00000004d8600000|100%| O| |TAMS 0x00000004d8600000, 0x00000004d8400000| Untracked +| 27|0x00000004d8600000, 0x00000004d8800000, 0x00000004d8800000|100%| O| |TAMS 0x00000004d8800000, 0x00000004d8600000| Untracked +| 28|0x00000004d8800000, 0x00000004d8a00000, 0x00000004d8a00000|100%| O| |TAMS 0x00000004d8a00000, 0x00000004d8800000| Untracked +| 29|0x00000004d8a00000, 0x00000004d8c00000, 0x00000004d8c00000|100%| O| |TAMS 0x00000004d8c00000, 0x00000004d8a00000| Untracked +| 30|0x00000004d8c00000, 0x00000004d8e00000, 0x00000004d8e00000|100%| O| |TAMS 0x00000004d8e00000, 0x00000004d8c00000| Untracked +| 31|0x00000004d8e00000, 0x00000004d9000000, 0x00000004d9000000|100%| O| |TAMS 0x00000004d9000000, 0x00000004d8e00000| Untracked +| 32|0x00000004d9000000, 0x00000004d9200000, 0x00000004d9200000|100%| O| |TAMS 0x00000004d9200000, 0x00000004d9000000| Untracked +| 33|0x00000004d9200000, 0x00000004d9400000, 0x00000004d9400000|100%| O| |TAMS 0x00000004d9400000, 0x00000004d9200000| Untracked +| 34|0x00000004d9400000, 0x00000004d9600000, 0x00000004d9600000|100%| O| |TAMS 0x00000004d9600000, 0x00000004d9400000| Untracked +| 35|0x00000004d9600000, 0x00000004d9800000, 0x00000004d9800000|100%| O| |TAMS 0x00000004d9800000, 0x00000004d9600000| Untracked +| 36|0x00000004d9800000, 0x00000004d9a00000, 0x00000004d9a00000|100%| O| |TAMS 0x00000004d9a00000, 0x00000004d9800000| Untracked +| 37|0x00000004d9a00000, 0x00000004d9c00000, 0x00000004d9c00000|100%| O| |TAMS 0x00000004d9c00000, 0x00000004d9a00000| Untracked +| 38|0x00000004d9c00000, 0x00000004d9e00000, 0x00000004d9e00000|100%| O| |TAMS 0x00000004d9e00000, 0x00000004d9c00000| Untracked +| 39|0x00000004d9e00000, 0x00000004da000000, 0x00000004da000000|100%| O| |TAMS 0x00000004da000000, 0x00000004d9e00000| Untracked +| 40|0x00000004da000000, 0x00000004da200000, 0x00000004da200000|100%| O| |TAMS 0x00000004da200000, 0x00000004da000000| Untracked +| 41|0x00000004da200000, 0x00000004da3c8200, 0x00000004da400000| 89%| O| |TAMS 0x00000004da3c8200, 0x00000004da200000| Untracked +| 42|0x00000004da400000, 0x00000004da600000, 0x00000004da600000|100%|HS| |TAMS 0x00000004da400000, 0x00000004da400000| Complete +| 43|0x00000004da600000, 0x00000004da800000, 0x00000004da800000|100%|HS| |TAMS 0x00000004da600000, 0x00000004da600000| Complete +| 44|0x00000004da800000, 0x00000004daa00000, 0x00000004daa00000|100%|HS| |TAMS 0x00000004da800000, 0x00000004da800000| Complete +| 45|0x00000004daa00000, 0x00000004dac00000, 0x00000004dac00000|100%|HS| |TAMS 0x00000004daa00000, 0x00000004daa00000| Complete +| 46|0x00000004dac00000, 0x00000004dae00000, 0x00000004dae00000|100%|HS| |TAMS 0x00000004dac00000, 0x00000004dac00000| Complete +| 47|0x00000004dae00000, 0x00000004db000000, 0x00000004db000000|100%|HS| |TAMS 0x00000004dae00000, 0x00000004dae00000| Complete +| 48|0x00000004db000000, 0x00000004db200000, 0x00000004db200000|100%|HS| |TAMS 0x00000004db000000, 0x00000004db000000| Complete +| 49|0x00000004db200000, 0x00000004db400000, 0x00000004db400000|100%| O| |TAMS 0x00000004db200000, 0x00000004db200000| Untracked +| 50|0x00000004db400000, 0x00000004db600000, 0x00000004db600000|100%| O| |TAMS 0x00000004db400000, 0x00000004db400000| Untracked +| 51|0x00000004db600000, 0x00000004db800000, 0x00000004db800000|100%|HS| |TAMS 0x00000004db600000, 0x00000004db600000| Complete +| 52|0x00000004db800000, 0x00000004dba00000, 0x00000004dba00000|100%|HS| |TAMS 0x00000004db800000, 0x00000004db800000| Complete +| 53|0x00000004dba00000, 0x00000004dbc00000, 0x00000004dbc00000|100%|HS| |TAMS 0x00000004dba00000, 0x00000004dba00000| Complete +| 54|0x00000004dbc00000, 0x00000004dbe00000, 0x00000004dbe00000|100%| O| |TAMS 0x00000004dbc00000, 0x00000004dbc00000| Untracked +| 55|0x00000004dbe00000, 0x00000004dc000000, 0x00000004dc000000|100%| O| |TAMS 0x00000004dbe00000, 0x00000004dbe00000| Untracked +| 56|0x00000004dc000000, 0x00000004dc200000, 0x00000004dc200000|100%| O| |TAMS 0x00000004dc000000, 0x00000004dc000000| Untracked +| 57|0x00000004dc200000, 0x00000004dc400000, 0x00000004dc400000|100%| O| |TAMS 0x00000004dc200000, 0x00000004dc200000| Untracked +| 58|0x00000004dc400000, 0x00000004dc600000, 0x00000004dc600000|100%|HS| |TAMS 0x00000004dc400000, 0x00000004dc400000| Complete +| 59|0x00000004dc600000, 0x00000004dc800000, 0x00000004dc800000|100%|HC| |TAMS 0x00000004dc600000, 0x00000004dc600000| Complete +| 60|0x00000004dc800000, 0x00000004dca00000, 0x00000004dca00000|100%|HC| |TAMS 0x00000004dc800000, 0x00000004dc800000| Complete +| 61|0x00000004dca00000, 0x00000004dca00000, 0x00000004dcc00000| 0%| F| |TAMS 0x00000004dca00000, 0x00000004dca00000| Untracked +| 62|0x00000004dcc00000, 0x00000004dcc00000, 0x00000004dce00000| 0%| F| |TAMS 0x00000004dcc00000, 0x00000004dcc00000| Untracked +| 63|0x00000004dce00000, 0x00000004dd000000, 0x00000004dd000000|100%| O| |TAMS 0x00000004dce00000, 0x00000004dce00000| Untracked +| 64|0x00000004dd000000, 0x00000004dd200000, 0x00000004dd200000|100%| O| |TAMS 0x00000004dd000000, 0x00000004dd000000| Untracked +| 65|0x00000004dd200000, 0x00000004dd291800, 0x00000004dd400000| 28%| O| |TAMS 0x00000004dd200000, 0x00000004dd200000| Untracked +| 66|0x00000004dd400000, 0x00000004dd400000, 0x00000004dd600000| 0%| F| |TAMS 0x00000004dd400000, 0x00000004dd400000| Untracked +| 67|0x00000004dd600000, 0x00000004dd600000, 0x00000004dd800000| 0%| F| |TAMS 0x00000004dd600000, 0x00000004dd600000| Untracked +| 68|0x00000004dd800000, 0x00000004dd800000, 0x00000004dda00000| 0%| F| |TAMS 0x00000004dd800000, 0x00000004dd800000| Untracked +| 69|0x00000004dda00000, 0x00000004dda00000, 0x00000004ddc00000| 0%| F| |TAMS 0x00000004dda00000, 0x00000004dda00000| Untracked +| 70|0x00000004ddc00000, 0x00000004ddc00000, 0x00000004dde00000| 0%| F| |TAMS 0x00000004ddc00000, 0x00000004ddc00000| Untracked +| 71|0x00000004dde00000, 0x00000004de000000, 0x00000004de000000|100%| O| |TAMS 0x00000004dde00000, 0x00000004dde00000| Untracked +| 72|0x00000004de000000, 0x00000004de200000, 0x00000004de200000|100%| O| |TAMS 0x00000004de000000, 0x00000004de000000| Untracked +| 73|0x00000004de200000, 0x00000004de400000, 0x00000004de400000|100%| O| |TAMS 0x00000004de200000, 0x00000004de200000| Untracked +| 74|0x00000004de400000, 0x00000004de600000, 0x00000004de600000|100%| O| |TAMS 0x00000004de400000, 0x00000004de400000| Untracked +| 75|0x00000004de600000, 0x00000004de800000, 0x00000004de800000|100%| O| |TAMS 0x00000004de600000, 0x00000004de600000| Untracked +| 76|0x00000004de800000, 0x00000004dea00000, 0x00000004dea00000|100%| O| |TAMS 0x00000004de800000, 0x00000004de800000| Untracked +| 77|0x00000004dea00000, 0x00000004dec00000, 0x00000004dec00000|100%| O| |TAMS 0x00000004dea00000, 0x00000004dea00000| Untracked +| 78|0x00000004dec00000, 0x00000004dee00000, 0x00000004dee00000|100%| O| |TAMS 0x00000004dec00000, 0x00000004dec00000| Untracked +| 79|0x00000004dee00000, 0x00000004dee00000, 0x00000004df000000| 0%| F| |TAMS 0x00000004dee00000, 0x00000004dee00000| Untracked +| 80|0x00000004df000000, 0x00000004df000000, 0x00000004df200000| 0%| F| |TAMS 0x00000004df000000, 0x00000004df000000| Untracked +| 81|0x00000004df200000, 0x00000004df200000, 0x00000004df400000| 0%| F| |TAMS 0x00000004df200000, 0x00000004df200000| Untracked +| 82|0x00000004df400000, 0x00000004df46bbf8, 0x00000004df600000| 21%| S|CS|TAMS 0x00000004df400000, 0x00000004df400000| Complete +| 83|0x00000004df600000, 0x00000004df600000, 0x00000004df800000| 0%| F| |TAMS 0x00000004df600000, 0x00000004df600000| Untracked +| 84|0x00000004df800000, 0x00000004df800000, 0x00000004dfa00000| 0%| F| |TAMS 0x00000004df800000, 0x00000004df800000| Untracked +| 85|0x00000004dfa00000, 0x00000004dfa00000, 0x00000004dfc00000| 0%| F| |TAMS 0x00000004dfa00000, 0x00000004dfa00000| Untracked +| 86|0x00000004dfc00000, 0x00000004dfe00000, 0x00000004dfe00000|100%| S|CS|TAMS 0x00000004dfc00000, 0x00000004dfc00000| Complete +| 87|0x00000004dfe00000, 0x00000004e0000000, 0x00000004e0000000|100%| S|CS|TAMS 0x00000004dfe00000, 0x00000004dfe00000| Complete +| 88|0x00000004e0000000, 0x00000004e0200000, 0x00000004e0200000|100%| S|CS|TAMS 0x00000004e0000000, 0x00000004e0000000| Complete +| 89|0x00000004e0200000, 0x00000004e0200000, 0x00000004e0400000| 0%| F| |TAMS 0x00000004e0200000, 0x00000004e0200000| Untracked +| 90|0x00000004e0400000, 0x00000004e0400000, 0x00000004e0600000| 0%| F| |TAMS 0x00000004e0400000, 0x00000004e0400000| Untracked +| 91|0x00000004e0600000, 0x00000004e0600000, 0x00000004e0800000| 0%| F| |TAMS 0x00000004e0600000, 0x00000004e0600000| Untracked +| 92|0x00000004e0800000, 0x00000004e0800000, 0x00000004e0a00000| 0%| F| |TAMS 0x00000004e0800000, 0x00000004e0800000| Untracked +| 93|0x00000004e0a00000, 0x00000004e0a00000, 0x00000004e0c00000| 0%| F| |TAMS 0x00000004e0a00000, 0x00000004e0a00000| Untracked +| 94|0x00000004e0c00000, 0x00000004e0c00000, 0x00000004e0e00000| 0%| F| |TAMS 0x00000004e0c00000, 0x00000004e0c00000| Untracked +| 95|0x00000004e0e00000, 0x00000004e0e00000, 0x00000004e1000000| 0%| F| |TAMS 0x00000004e0e00000, 0x00000004e0e00000| Untracked +| 96|0x00000004e1000000, 0x00000004e1000000, 0x00000004e1200000| 0%| F| |TAMS 0x00000004e1000000, 0x00000004e1000000| Untracked +| 97|0x00000004e1200000, 0x00000004e1200000, 0x00000004e1400000| 0%| F| |TAMS 0x00000004e1200000, 0x00000004e1200000| Untracked +| 98|0x00000004e1400000, 0x00000004e1400000, 0x00000004e1600000| 0%| F| |TAMS 0x00000004e1400000, 0x00000004e1400000| Untracked +| 99|0x00000004e1600000, 0x00000004e1600000, 0x00000004e1800000| 0%| F| |TAMS 0x00000004e1600000, 0x00000004e1600000| Untracked +| 100|0x00000004e1800000, 0x00000004e1800000, 0x00000004e1a00000| 0%| F| |TAMS 0x00000004e1800000, 0x00000004e1800000| Untracked +| 101|0x00000004e1a00000, 0x00000004e1a00000, 0x00000004e1c00000| 0%| F| |TAMS 0x00000004e1a00000, 0x00000004e1a00000| Untracked +| 102|0x00000004e1c00000, 0x00000004e1c00000, 0x00000004e1e00000| 0%| F| |TAMS 0x00000004e1c00000, 0x00000004e1c00000| Untracked +| 103|0x00000004e1e00000, 0x00000004e1e00000, 0x00000004e2000000| 0%| F| |TAMS 0x00000004e1e00000, 0x00000004e1e00000| Untracked +| 104|0x00000004e2000000, 0x00000004e2000000, 0x00000004e2200000| 0%| F| |TAMS 0x00000004e2000000, 0x00000004e2000000| Untracked +| 105|0x00000004e2200000, 0x00000004e2200000, 0x00000004e2400000| 0%| F| |TAMS 0x00000004e2200000, 0x00000004e2200000| Untracked +| 106|0x00000004e2400000, 0x00000004e2400000, 0x00000004e2600000| 0%| F| |TAMS 0x00000004e2400000, 0x00000004e2400000| Untracked +| 107|0x00000004e2600000, 0x00000004e2600000, 0x00000004e2800000| 0%| F| |TAMS 0x00000004e2600000, 0x00000004e2600000| Untracked +| 108|0x00000004e2800000, 0x00000004e2800000, 0x00000004e2a00000| 0%| F| |TAMS 0x00000004e2800000, 0x00000004e2800000| Untracked +| 109|0x00000004e2a00000, 0x00000004e2a00000, 0x00000004e2c00000| 0%| F| |TAMS 0x00000004e2a00000, 0x00000004e2a00000| Untracked +| 110|0x00000004e2c00000, 0x00000004e2d3c128, 0x00000004e2e00000| 61%| E| |TAMS 0x00000004e2c00000, 0x00000004e2c00000| Complete +| 111|0x00000004e2e00000, 0x00000004e3000000, 0x00000004e3000000|100%| E|CS|TAMS 0x00000004e2e00000, 0x00000004e2e00000| Complete +| 112|0x00000004e3000000, 0x00000004e3200000, 0x00000004e3200000|100%| E|CS|TAMS 0x00000004e3000000, 0x00000004e3000000| Complete +| 113|0x00000004e3200000, 0x00000004e3400000, 0x00000004e3400000|100%| E|CS|TAMS 0x00000004e3200000, 0x00000004e3200000| Complete +| 114|0x00000004e3400000, 0x00000004e3600000, 0x00000004e3600000|100%| E|CS|TAMS 0x00000004e3400000, 0x00000004e3400000| Complete +| 115|0x00000004e3600000, 0x00000004e3800000, 0x00000004e3800000|100%| E|CS|TAMS 0x00000004e3600000, 0x00000004e3600000| Complete +| 116|0x00000004e3800000, 0x00000004e3a00000, 0x00000004e3a00000|100%| E|CS|TAMS 0x00000004e3800000, 0x00000004e3800000| Complete +| 117|0x00000004e3a00000, 0x00000004e3c00000, 0x00000004e3c00000|100%| E|CS|TAMS 0x00000004e3a00000, 0x00000004e3a00000| Complete +| 118|0x00000004e3c00000, 0x00000004e3e00000, 0x00000004e3e00000|100%| E|CS|TAMS 0x00000004e3c00000, 0x00000004e3c00000| Complete +| 119|0x00000004e3e00000, 0x00000004e4000000, 0x00000004e4000000|100%| E|CS|TAMS 0x00000004e3e00000, 0x00000004e3e00000| Complete +| 120|0x00000004e4000000, 0x00000004e4200000, 0x00000004e4200000|100%| E|CS|TAMS 0x00000004e4000000, 0x00000004e4000000| Complete +| 121|0x00000004e4200000, 0x00000004e4400000, 0x00000004e4400000|100%| E|CS|TAMS 0x00000004e4200000, 0x00000004e4200000| Complete +| 122|0x00000004e4400000, 0x00000004e4600000, 0x00000004e4600000|100%| E|CS|TAMS 0x00000004e4400000, 0x00000004e4400000| Complete +| 123|0x00000004e4600000, 0x00000004e4800000, 0x00000004e4800000|100%| E|CS|TAMS 0x00000004e4600000, 0x00000004e4600000| Complete +| 124|0x00000004e4800000, 0x00000004e4a00000, 0x00000004e4a00000|100%| E|CS|TAMS 0x00000004e4800000, 0x00000004e4800000| Complete +| 125|0x00000004e4a00000, 0x00000004e4c00000, 0x00000004e4c00000|100%| E|CS|TAMS 0x00000004e4a00000, 0x00000004e4a00000| Complete +| 126|0x00000004e4c00000, 0x00000004e4e00000, 0x00000004e4e00000|100%| E|CS|TAMS 0x00000004e4c00000, 0x00000004e4c00000| Complete +| 127|0x00000004e4e00000, 0x00000004e5000000, 0x00000004e5000000|100%| E|CS|TAMS 0x00000004e4e00000, 0x00000004e4e00000| Complete +| 128|0x00000004e5000000, 0x00000004e5200000, 0x00000004e5200000|100%| E|CS|TAMS 0x00000004e5000000, 0x00000004e5000000| Complete +| 129|0x00000004e5200000, 0x00000004e5400000, 0x00000004e5400000|100%| E|CS|TAMS 0x00000004e5200000, 0x00000004e5200000| Complete +| 130|0x00000004e5400000, 0x00000004e5600000, 0x00000004e5600000|100%| E|CS|TAMS 0x00000004e5400000, 0x00000004e5400000| Complete +| 131|0x00000004e5600000, 0x00000004e5800000, 0x00000004e5800000|100%| E|CS|TAMS 0x00000004e5600000, 0x00000004e5600000| Complete +| 132|0x00000004e5800000, 0x00000004e5a00000, 0x00000004e5a00000|100%| E|CS|TAMS 0x00000004e5800000, 0x00000004e5800000| Complete +| 133|0x00000004e5a00000, 0x00000004e5c00000, 0x00000004e5c00000|100%| E|CS|TAMS 0x00000004e5a00000, 0x00000004e5a00000| Complete +| 134|0x00000004e5c00000, 0x00000004e5e00000, 0x00000004e5e00000|100%| E|CS|TAMS 0x00000004e5c00000, 0x00000004e5c00000| Complete +| 135|0x00000004e5e00000, 0x00000004e6000000, 0x00000004e6000000|100%| E|CS|TAMS 0x00000004e5e00000, 0x00000004e5e00000| Complete +| 136|0x00000004e6000000, 0x00000004e6200000, 0x00000004e6200000|100%| E|CS|TAMS 0x00000004e6000000, 0x00000004e6000000| Complete +| 137|0x00000004e6200000, 0x00000004e6400000, 0x00000004e6400000|100%| E|CS|TAMS 0x00000004e6200000, 0x00000004e6200000| Complete +| 138|0x00000004e6400000, 0x00000004e6600000, 0x00000004e6600000|100%| E|CS|TAMS 0x00000004e6400000, 0x00000004e6400000| Complete +| 139|0x00000004e6600000, 0x00000004e6800000, 0x00000004e6800000|100%| E|CS|TAMS 0x00000004e6600000, 0x00000004e6600000| Complete +| 140|0x00000004e6800000, 0x00000004e6a00000, 0x00000004e6a00000|100%| E|CS|TAMS 0x00000004e6800000, 0x00000004e6800000| Complete +| 141|0x00000004e6a00000, 0x00000004e6c00000, 0x00000004e6c00000|100%| E|CS|TAMS 0x00000004e6a00000, 0x00000004e6a00000| Complete +| 142|0x00000004e6c00000, 0x00000004e6e00000, 0x00000004e6e00000|100%| E|CS|TAMS 0x00000004e6c00000, 0x00000004e6c00000| Complete +| 143|0x00000004e6e00000, 0x00000004e7000000, 0x00000004e7000000|100%| E|CS|TAMS 0x00000004e6e00000, 0x00000004e6e00000| Complete +| 144|0x00000004e7000000, 0x00000004e7200000, 0x00000004e7200000|100%| E|CS|TAMS 0x00000004e7000000, 0x00000004e7000000| Complete +| 145|0x00000004e7200000, 0x00000004e7400000, 0x00000004e7400000|100%| E|CS|TAMS 0x00000004e7200000, 0x00000004e7200000| Complete +| 146|0x00000004e7400000, 0x00000004e7600000, 0x00000004e7600000|100%| E|CS|TAMS 0x00000004e7400000, 0x00000004e7400000| Complete +| 147|0x00000004e7600000, 0x00000004e7800000, 0x00000004e7800000|100%| E|CS|TAMS 0x00000004e7600000, 0x00000004e7600000| Complete +| 148|0x00000004e7800000, 0x00000004e7a00000, 0x00000004e7a00000|100%| E|CS|TAMS 0x00000004e7800000, 0x00000004e7800000| Complete +| 149|0x00000004e7a00000, 0x00000004e7c00000, 0x00000004e7c00000|100%| E|CS|TAMS 0x00000004e7a00000, 0x00000004e7a00000| Complete +| 150|0x00000004e7c00000, 0x00000004e7e00000, 0x00000004e7e00000|100%| E|CS|TAMS 0x00000004e7c00000, 0x00000004e7c00000| Complete +| 151|0x00000004e7e00000, 0x00000004e8000000, 0x00000004e8000000|100%| E|CS|TAMS 0x00000004e7e00000, 0x00000004e7e00000| Complete +| 152|0x00000004e8000000, 0x00000004e8200000, 0x00000004e8200000|100%| E|CS|TAMS 0x00000004e8000000, 0x00000004e8000000| Complete +| 153|0x00000004e8200000, 0x00000004e8400000, 0x00000004e8400000|100%| E|CS|TAMS 0x00000004e8200000, 0x00000004e8200000| Complete +| 154|0x00000004e8400000, 0x00000004e8600000, 0x00000004e8600000|100%| E|CS|TAMS 0x00000004e8400000, 0x00000004e8400000| Complete +| 155|0x00000004e8600000, 0x00000004e8800000, 0x00000004e8800000|100%| E|CS|TAMS 0x00000004e8600000, 0x00000004e8600000| Complete +| 156|0x00000004e8800000, 0x00000004e8a00000, 0x00000004e8a00000|100%| E|CS|TAMS 0x00000004e8800000, 0x00000004e8800000| Complete +| 157|0x00000004e8a00000, 0x00000004e8c00000, 0x00000004e8c00000|100%| E|CS|TAMS 0x00000004e8a00000, 0x00000004e8a00000| Complete +| 158|0x00000004e8c00000, 0x00000004e8e00000, 0x00000004e8e00000|100%| E|CS|TAMS 0x00000004e8c00000, 0x00000004e8c00000| Complete +| 339|0x00000004ff600000, 0x00000004ff800000, 0x00000004ff800000|100%| E|CS|TAMS 0x00000004ff600000, 0x00000004ff600000| Complete +| 340|0x00000004ff800000, 0x00000004ffa00000, 0x00000004ffa00000|100%| E|CS|TAMS 0x00000004ff800000, 0x00000004ff800000| Complete +| 440|0x000000050c000000, 0x000000050c200000, 0x000000050c200000|100%| E|CS|TAMS 0x000000050c000000, 0x000000050c000000| Complete +| 441|0x000000050c200000, 0x000000050c400000, 0x000000050c400000|100%| E|CS|TAMS 0x000000050c200000, 0x000000050c200000| Complete +| 442|0x000000050c400000, 0x000000050c600000, 0x000000050c600000|100%| E|CS|TAMS 0x000000050c400000, 0x000000050c400000| Complete +| 767|0x0000000534e00000, 0x0000000535000000, 0x0000000535000000|100%| E|CS|TAMS 0x0000000534e00000, 0x0000000534e00000| Complete +|2046|0x00000005d4c00000, 0x00000005d4d74000, 0x00000005d4e00000| 72%|OA| |TAMS 0x00000005d4d74000, 0x00000005d4c00000| Untracked +|2047|0x00000005d4e00000, 0x00000005d4e74000, 0x00000005d5000000| 22%|CA| |TAMS 0x00000005d4e74000, 0x00000005d4e00000| Untracked + +Card table byte_map: [0x0000000107774000,0x0000000107f74000] _byte_map_base: 0x00000001050cc000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x0000000102b9e8c0, (CMBitMap*) 0x0000000102b9e900 + Prev Bits: [0x00000001112d0000, 0x00000001152d0000) + Next Bits: [0x00000001152d0000, 0x00000001192d0000) + +Polling page: 0x0000000102340000 + +Metaspace: + +Usage: + Non-class: 130.13 MB used. + Class: 17.80 MB used. + Both: 147.93 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 130.81 MB ( 68%) committed, 3 nodes. + Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 149.19 MB ( 12%) committed. + +Chunk freelists: + Non-Class: 12.98 MB + Class: 13.67 MB + Both: 26.65 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 164.69 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 12. +num_arena_births: 2070. +num_arena_deaths: 0. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2384. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 12. +num_chunks_taken_from_freelist: 7802. +num_chunk_merges: 12. +num_chunk_splits: 5805. +num_chunks_enlarged: 4373. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=9105Kb max_used=9105Kb free=110062Kb + bounds [0x00000001241a4000, 0x0000000124a94000, 0x000000012b604000] +CodeHeap 'profiled nmethods': size=119152Kb used=20927Kb max_used=20927Kb free=98225Kb + bounds [0x000000011c604000, 0x000000011da74000, 0x0000000123a60000] +CodeHeap 'non-nmethods': size=7440Kb used=3322Kb max_used=3446Kb free=4117Kb + bounds [0x0000000123a60000, 0x0000000123dd0000, 0x00000001241a4000] + total_blobs=12431 nmethods=11369 adapters=973 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 21.548 Thread 0x000000093dbbb000 nmethod 14855 0x000000011da66990 code [0x000000011da66b80, 0x000000011da66e28] +Event: 21.548 Thread 0x000000093dbbb000 14853 3 org.apache.comet.serde.Config$ConfigMap$Builder::internalGetMutableEntries (110 bytes) +Event: 21.548 Thread 0x000000093dbbaa00 nmethod 14857 0x000000011da66f90 code [0x000000011da67180, 0x000000011da67438] +Event: 21.548 Thread 0x000000093dbbaa00 14860 3 org.apache.spark.sql.internal.SQLConf$$Lambda$514/0x000000e0013364f8:: (20 bytes) +Event: 21.548 Thread 0x000000093dbbaa00 nmethod 14860 0x000000011da67610 code [0x000000011da677c0, 0x000000011da67a18] +Event: 21.548 Thread 0x000000093dbbaa00 14861 3 org.apache.spark.sql.internal.SQLConf::getConfString (50 bytes) +Event: 21.548 Thread 0x000000093dbbb600 nmethod 14850 0x000000011da67b10 code [0x000000011da67d40, 0x000000011da682c8] +Event: 21.548 Thread 0x000000093dbbb600 14862 3 scala.collection.immutable.List::apply (15 bytes) +Event: 21.548 Thread 0x000000093dbbb000 nmethod 14853 0x000000011da68590 code [0x000000011da687c0, 0x000000011da68d58] +Event: 21.548 Thread 0x000000093dbbb000 14863 3 org.apache.logging.log4j.core.pattern.MessagePatternConverter$SimpleMessagePatternConverter::format (87 bytes) +Event: 21.548 Thread 0x000000093dbbb600 nmethod 14862 0x000000011da68f10 code [0x000000011da690c0, 0x000000011da692b8] +Event: 21.548 Thread 0x000000093e8d8000 14864 4 org.apache.spark.internal.config.ConfigReader::$jacocoInit (53 bytes) +Event: 21.548 Thread 0x000000093dbbb600 14865 3 org.apache.comet.ConfigEntry::key (13 bytes) +Event: 21.548 Thread 0x000000093dbbb600 nmethod 14865 0x000000011da69410 code [0x000000011da695c0, 0x000000011da69748] +Event: 21.548 Thread 0x000000093dbbb000 nmethod 14863 0x000000011da69890 code [0x000000011da69b00, 0x000000011da6a408] +Event: 21.549 Thread 0x000000093e8d8000 nmethod 14864 0x0000000124a6d790 code [0x0000000124a6d940, 0x0000000124a6d9e0] +Event: 21.549 Thread 0x000000093e8d8000 14866 4 scala.collection.mutable.HashTable$::nextPositivePowerOfTwo (19 bytes) +Event: 21.549 Thread 0x000000093dbbaa00 nmethod 14861 0x000000011da6a710 code [0x000000011da6aa00, 0x000000011da6b538] +Event: 21.549 Thread 0x000000093e872a00 nmethod 14854 0x000000011da6b990 code [0x000000011da6c040, 0x000000011da6f468] +Event: 21.549 Thread 0x000000093e8d8000 nmethod 14866 0x0000000124a6da90 code [0x0000000124a6dc40, 0x0000000124a6dd30] + +GC Heap History (20 events): +Event: 6.529 GC heap before +{Heap before GC invocations=25 (full 0): + garbage-first heap total 342016K, used 286067K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 87 young (178176K), 13 survivors (26624K) + Metaspace used 123550K, committed 124416K, reserved 1179648K + class space used 14868K, committed 15296K, reserved 1048576K +} +Event: 6.532 GC heap after +{Heap after GC invocations=26 (full 0): + garbage-first heap total 342016K, used 147927K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 6 young (12288K), 6 survivors (12288K) + Metaspace used 123550K, committed 124416K, reserved 1179648K + class space used 14868K, committed 15296K, reserved 1048576K +} +Event: 6.821 GC heap before +{Heap before GC invocations=26 (full 0): + garbage-first heap total 342016K, used 276951K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 70 young (143360K), 6 survivors (12288K) + Metaspace used 127470K, committed 128384K, reserved 1179648K + class space used 15125K, committed 15552K, reserved 1048576K +} +Event: 6.823 GC heap after +{Heap after GC invocations=27 (full 0): + garbage-first heap total 342016K, used 147833K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 6 young (12288K), 6 survivors (12288K) + Metaspace used 127470K, committed 128384K, reserved 1179648K + class space used 15125K, committed 15552K, reserved 1048576K +} +Event: 6.992 GC heap before +{Heap before GC invocations=27 (full 0): + garbage-first heap total 342016K, used 291193K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 71 young (145408K), 6 survivors (12288K) + Metaspace used 133222K, committed 134144K, reserved 1179648K + class space used 15928K, committed 16384K, reserved 1048576K +} +Event: 6.993 GC heap after +{Heap after GC invocations=28 (full 0): + garbage-first heap total 342016K, used 134137K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 133222K, committed 134144K, reserved 1179648K + class space used 15928K, committed 16384K, reserved 1048576K +} +Event: 7.258 GC heap before +{Heap before GC invocations=28 (full 0): + garbage-first heap total 342016K, used 289785K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 74 young (151552K), 2 survivors (4096K) + Metaspace used 139905K, committed 140864K, reserved 1179648K + class space used 16753K, committed 17216K, reserved 1048576K +} +Event: 7.259 GC heap after +{Heap after GC invocations=29 (full 0): + garbage-first heap total 342016K, used 135614K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 139905K, committed 140864K, reserved 1179648K + class space used 16753K, committed 17216K, reserved 1048576K +} +Event: 7.470 GC heap before +{Heap before GC invocations=29 (full 0): + garbage-first heap total 342016K, used 289214K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 76 young (155648K), 3 survivors (6144K) + Metaspace used 147625K, committed 148736K, reserved 1179648K + class space used 17607K, committed 18112K, reserved 1048576K +} +Event: 7.472 GC heap after +{Heap after GC invocations=30 (full 0): + garbage-first heap total 342016K, used 137564K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 147625K, committed 148736K, reserved 1179648K + class space used 17607K, committed 18112K, reserved 1048576K +} +Event: 7.851 GC heap before +{Heap before GC invocations=30 (full 0): + garbage-first heap total 342016K, used 285020K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 76 young (155648K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 7.853 GC heap after +{Heap after GC invocations=31 (full 0): + garbage-first heap total 342016K, used 137850K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.168 GC heap before +{Heap before GC invocations=31 (full 0): + garbage-first heap total 342016K, used 289402K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 78 young (159744K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.170 GC heap after +{Heap after GC invocations=32 (full 0): + garbage-first heap total 342016K, used 137747K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.497 GC heap before +{Heap before GC invocations=32 (full 0): + garbage-first heap total 342016K, used 291347K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 79 young (161792K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.498 GC heap after +{Heap after GC invocations=33 (full 0): + garbage-first heap total 342016K, used 137753K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.814 GC heap before +{Heap before GC invocations=33 (full 0): + garbage-first heap total 342016K, used 293401K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 80 young (163840K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 8.815 GC heap after +{Heap after GC invocations=34 (full 0): + garbage-first heap total 342016K, used 137757K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 9.136 GC heap before +{Heap before GC invocations=34 (full 0): + garbage-first heap total 342016K, used 293405K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 80 young (163840K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} +Event: 9.138 GC heap after +{Heap after GC invocations=35 (full 0): + garbage-first heap total 342016K, used 137909K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 148901K, committed 150016K, reserved 1245184K + class space used 17797K, committed 18304K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.418 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li +Event: 3.418 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.418 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.403 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-6348563280641709946.dylib +Event: 5.871 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 5.871 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 5.871 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 5.871 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 5.871 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 5.871 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 5.871 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.871 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 5.871 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 5.872 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 5.872 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 6.011 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-1210006264174898416.dylib +Event: 6.749 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-5955df27-d344-4639-975a-b5e107a03f33-libsnappyjava.dylib + +Deoptimization events (20 events): +Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000124a380fc relative=0x0000000000000fbc +Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000124a380fc method=scala.collection.TraversableOnce.sum(Lscala/math/Numeric;)Ljava/lang/Object; @ 6 c2 +Event: 21.539 Thread 0x000000093d9d8600 DEOPT PACKING pc=0x0000000124a380fc sp=0x000000017ea89090 +Event: 21.539 Thread 0x000000093d9d8600 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ea88f90 mode 2 +Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c +Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 21.539 Thread 0x000000093d9d8600 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017ea88ef0 +Event: 21.539 Thread 0x000000093d9d8600 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ea88df0 mode 2 +Event: 21.539 Thread 0x000000093d9d8000 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c +Event: 21.539 Thread 0x000000093d9d8000 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 21.539 Thread 0x000000093d9d9200 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c +Event: 21.539 Thread 0x000000093d9d8000 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017e67cf50 +Event: 21.539 Thread 0x000000093d9d9200 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 21.539 Thread 0x000000093d9d9200 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017f2a0f50 +Event: 21.539 Thread 0x000000093d9d8000 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017e67ce50 mode 2 +Event: 21.539 Thread 0x000000093d9d9200 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017f2a0e50 mode 2 +Event: 21.539 Thread 0x000000093d9d8c00 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b06a8 relative=0x00000000000000e8 +Event: 21.539 Thread 0x000000093d9d8c00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b06a8 method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 +Event: 21.539 Thread 0x000000093d9d8c00 DEOPT PACKING pc=0x00000001249b06a8 sp=0x000000017ee94ed0 +Event: 21.539 Thread 0x000000093d9d8c00 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ee94e50 mode 2 + +Classes loaded (20 events): +Event: 7.438 Loading class java/lang/Class$EnclosingMethodInfo +Event: 7.438 Loading class java/lang/Class$EnclosingMethodInfo done +Event: 15.240 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper +Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$Unmapper +Event: 15.241 Loading class jdk/internal/access/foreign/UnmapperProxy +Event: 15.241 Loading class jdk/internal/access/foreign/UnmapperProxy done +Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$Unmapper done +Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done +Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask +Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask +Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done +Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done +Event: 20.226 Loading class java/util/function/LongBinaryOperator +Event: 20.226 Loading class java/util/function/LongBinaryOperator done +Event: 21.429 Loading class sun/nio/ch/ChannelInputStream +Event: 21.429 Loading class sun/nio/ch/ChannelInputStream done +Event: 21.482 Loading class java/math/BigDecimal$StringBuilderHelper +Event: 21.482 Loading class java/math/BigDecimal$StringBuilderHelper done +Event: 21.544 Loading class java/nio/channels/Channels$ReadableByteChannelImpl +Event: 21.544 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 21.540 Thread 0x000000093d9d9200 Exception (0x00000004e33ed9f0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.540 Thread 0x000000093d9d8c00 Exception (0x00000004e388ac00) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.540 Thread 0x000000093d9d8000 Exception (0x00000004e35ba828) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.540 Thread 0x000000093d9d8600 Exception (0x00000004e33737a8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.540 Thread 0x000000093d9bf600 Exception (0x00000004e32de4e0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e30134a0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e241b0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e3384178) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32f2ca8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3152498) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e28030) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e3387f50) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e3017478) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3156270) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32f6a80) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e2b910) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e338b7e0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e301ad08) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3159b00) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32fa310) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] + +VM Operations (20 events): +Event: 8.168 Executing VM operation: G1CollectForAllocation +Event: 8.170 Executing VM operation: G1CollectForAllocation done +Event: 8.497 Executing VM operation: G1CollectForAllocation +Event: 8.498 Executing VM operation: G1CollectForAllocation done +Event: 8.814 Executing VM operation: G1CollectForAllocation +Event: 8.815 Executing VM operation: G1CollectForAllocation done +Event: 9.136 Executing VM operation: G1CollectForAllocation +Event: 9.138 Executing VM operation: G1CollectForAllocation done +Event: 12.153 Executing VM operation: Cleanup +Event: 12.153 Executing VM operation: Cleanup done +Event: 16.173 Executing VM operation: Cleanup +Event: 16.173 Executing VM operation: Cleanup done +Event: 21.198 Executing VM operation: Cleanup +Event: 21.198 Executing VM operation: Cleanup done +Event: 21.472 Executing VM operation: ICBufferFull +Event: 21.472 Executing VM operation: ICBufferFull done +Event: 21.508 Executing VM operation: ICBufferFull +Event: 21.508 Executing VM operation: ICBufferFull done +Event: 21.548 Executing VM operation: ICBufferFull +Event: 21.548 Executing VM operation: ICBufferFull done + +Memory protections (20 events): +Event: 7.060 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 3 +Event: 7.060 Protecting memory [0x0000000300410000,0x000000030041c000] with protection modes 3 +Event: 7.266 Protecting memory [0x0000000300410000,0x000000030041c000] with protection modes 0 +Event: 7.269 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 0 +Event: 7.269 Protecting memory [0x0000000300c28000,0x0000000300c34000] with protection modes 0 +Event: 7.270 Protecting memory [0x0000000301034000,0x0000000301040000] with protection modes 0 +Event: 7.270 Protecting memory [0x0000000301440000,0x000000030144c000] with protection modes 0 +Event: 9.274 Protecting memory [0x0000000171164000,0x0000000171170000] with protection modes 3 +Event: 14.280 Protecting memory [0x00000001748d0000,0x00000001748dc000] with protection modes 3 +Event: 15.240 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 3 +Event: 20.228 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 0 +Event: 21.428 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 3 +Event: 21.428 Protecting memory [0x0000000170d4c000,0x0000000170d58000] with protection modes 3 +Event: 21.432 Protecting memory [0x0000000170d4c000,0x0000000170d58000] with protection modes 0 +Event: 21.447 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 0 +Event: 21.447 Protecting memory [0x0000000171164000,0x0000000171170000] with protection modes 0 +Event: 21.447 Protecting memory [0x00000001748d0000,0x00000001748dc000] with protection modes 0 +Event: 21.460 Protecting memory [0x0000000302288000,0x0000000302294000] with protection modes 0 +Event: 21.472 Protecting memory [0x00000003026a0000,0x00000003026ac000] with protection modes 0 +Event: 21.527 Protecting memory [0x0000000302aac000,0x0000000302ab8000] with protection modes 0 + +Nmethod flushes (20 events): +Event: 7.445 Thread 0x000000093e873000 flushing nmethod 0x0000000124854590 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5dfd90 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f2410 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f2890 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f4190 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f5010 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f5e90 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f9310 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f9890 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5fae90 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5fee90 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d601690 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d603790 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d604810 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d605690 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d60a210 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d60e910 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d610610 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d613490 +Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d61fd10 + +Events (20 events): +Event: 7.060 Thread 0x000000093da19800 Thread exited: 0x000000093da19800 +Event: 7.060 Thread 0x000000093da19200 Thread exited: 0x000000093da19200 +Event: 7.266 Thread 0x000000093e752400 Thread added: 0x000000093db16400 +Event: 7.269 Thread 0x000000093db16400 Thread added: 0x000000093db17000 +Event: 7.269 Thread 0x000000093db16400 Thread added: 0x000000093db17600 +Event: 7.270 Thread 0x000000093db17600 Thread added: 0x000000093db68000 +Event: 7.270 Thread 0x000000093db68000 Thread added: 0x000000093db68600 +Event: 9.274 Thread 0x000000093d9bd800 Thread exited: 0x000000093d9bd800 +Event: 14.280 Thread 0x000000093d9d9800 Thread exited: 0x000000093d9d9800 +Event: 15.240 Thread 0x000000093e8d8600 Thread exited: 0x000000093e8d8600 +Event: 20.228 Thread 0x000000093e872a00 Thread added: 0x000000093dbba400 +Event: 21.428 Thread 0x000000093dbba400 Thread exited: 0x000000093dbba400 +Event: 21.428 Thread 0x000000093e8d8000 Thread exited: 0x000000093e8d8000 +Event: 21.432 Thread 0x000000093e872a00 Thread added: 0x000000093dbba400 +Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbaa00 +Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbb000 +Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbb600 +Event: 21.460 Thread 0x000000093dbbb600 Thread added: 0x000000093e8d8000 +Event: 21.472 Thread 0x000000093e785e00 Thread added: 0x000000093db6b600 +Event: 21.527 Thread 0x000000093e872400 Thread added: 0x000000093b2acc00 + + +Dynamic libraries: +0x0000000102358000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x0000000107f8c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x0000000102380000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x00000001023b0000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x0000000102400000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x000000011b3f8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x000000010266c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x000000011b444000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x000000011b420000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x000000011b4e8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x000000011b4fc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x000000011b510000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x000000011b524000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000147dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-6348563280641709946.dylib +0x000000011b564000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-1210006264174898416.dylib +0x000000011b540000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-5955df27-d344-4639-975a-b5e107a03f33-libsnappyjava.dylib + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:41 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 8.44 7.59 5.66 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(3784624k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/hs_err_pid59395.log b/spark/hs_err_pid59395.log new file mode 100644 index 0000000000..32c9dedeb3 --- /dev/null +++ b/spark/hs_err_pid59395.log @@ -0,0 +1,1886 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x0000000108215530, pid=59395, tid=66563 +# +# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) +# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) +# Problematic frame: +# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +# +# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again +# +# If you would like to submit a bug report, please visit: +# https://github.com/adoptium/adoptium-support/issues +# + +--------------- S U M M A R Y ------------ + +Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. + +Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) +Time: Sun Apr 12 09:00:35 2026 MDT elapsed time: 21.311638 seconds (0d 0h 0m 21s) + +--------------- T H R E A D --------------- + +Current thread (0x0000000afb041e00): JavaThread "Executor task launch worker for task 29.0 in stage 4.0 (TID 40)" daemon [_thread_in_vm, id=66563, stack(0x000000017ec70000,0x000000017f073000)] + +Stack: [0x000000017ec70000,0x000000017f073000], sp=0x000000017f06ac00, free space=4075k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 +V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 +C [libcomet-766714755951389016.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 +C [libcomet-766714755951389016.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc +C [libcomet-766714755951389016.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 +C [libcomet-766714755951389016.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 +C [libcomet-766714755951389016.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c +C [libcomet-766714755951389016.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 +C [libcomet-766714755951389016.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 +C [libcomet-766714755951389016.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 +C [libcomet-766714755951389016.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 +C [libcomet-766714755951389016.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 +C [libcomet-766714755951389016.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 +C [libcomet-766714755951389016.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 +C [libcomet-766714755951389016.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c +C [libcomet-766714755951389016.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 +C [libcomet-766714755951389016.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 +C [libcomet-766714755951389016.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c +C [libcomet-766714755951389016.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 +C [libcomet-766714755951389016.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 +C [libcomet-766714755951389016.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 +C [libcomet-766714755951389016.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c +C [libcomet-766714755951389016.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 +C [libcomet-766714755951389016.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 +C [libcomet-766714755951389016.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 +C [libcomet-766714755951389016.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c +C [libcomet-766714755951389016.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c +C [libcomet-766714755951389016.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c +C [libcomet-766714755951389016.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc +C [libcomet-766714755951389016.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 +C [libcomet-766714755951389016.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 +C [libcomet-766714755951389016.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 +C [libcomet-766714755951389016.dylib+0x106628] __rust_try+0x20 +C [libcomet-766714755951389016.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 +C [libcomet-766714755951389016.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 +C [libcomet-766714755951389016.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac +C [libcomet-766714755951389016.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3957+0x000000d002198000.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3950+0x000000d0021966b0.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2515+0x000000d001df15b8.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub +V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 +V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c +V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 +V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 +V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 +V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc +V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 +C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 +j org.apache.comet.CometExecIterator$$Lambda$3957+0x000000d002198000.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 +j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 +j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 +j org.apache.comet.CometExecIterator$$Lambda$3950+0x000000d0021966b0.apply()Ljava/lang/Object;+8 +j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 +j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 +j org.apache.comet.CometExecIterator.hasNext()Z+197 +j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 +j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 +j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 +j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 +j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 +j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 +j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2515+0x000000d001df15b8.apply()Ljava/lang/Object;+8 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 +j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 +j org.apache.spark.executor.Executor$TaskRunner.run()V+797 +j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 +j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 +j java.lang.Thread.run()V+11 java.base@17.0.17 +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 + +Registers: + x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000b069bc1f0 + x4=0x000000017f06ad20 x5=0x0000000afb041e00 x6=0x0000000b025d7b00 x7=0xfffff0003ffff800 + x8=0x000000d00217f978 x9=0x0000000000000000 x10=0x000000d000000000 x11=0x0000000000000004 +x12=0x0000000137020e48 x13=0x0000000000000000 x14=0xfffffffffffdb114 x15=0x0000000b025cc000 +x16=0x0000000000000040 x17=0x0000000b02016290 x18=0x0000000000000000 x19=0x0000000afb041e00 +x20=0x000000017f06ad50 x21=0x000000017f06ad20 x22=0x0000000afbc68018 x23=0x00000005016d06b0 +x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000b02033700 x27=0x0000000000000000 +x28=0x0000000000000001 fp=0x000000017f06ad10 lr=0x000000010821552c sp=0x000000017f06ac00 +pc=0x0000000108215530 cpsr=0x0000000000001000 + +Register to memory mapping: + + x0=0x0 is NULL + x1=0x0000000000000007 is an unknown value + x2=0x0000000000000001 is an unknown value + x3=0x0000000b069bc1f0 points into unknown readable memory: 0x0000000137020e48 | 48 0e 02 37 01 00 00 00 + x4=0x000000017f06ad20 is pointing into the stack for thread: 0x0000000afb041e00 + x5=0x0000000afb041e00 is a thread + x6=0x0000000b025d7b00 points into unknown readable memory: 0x0000000af564c600 | 00 c6 64 f5 0a 00 00 00 + x7=0xfffff0003ffff800 is an unknown value + x8=0x000000d00217f978 is pointing into metadata + x9=0x0 is NULL +x10=0x000000d000000000 is pointing into metadata +x11=0x0000000000000004 is an unknown value +x12={method} {0x0000000137020e48} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' +x13=0x0 is NULL +x14=0xfffffffffffdb114 is an unknown value +x15=0x0000000b025cc000 points into unknown readable memory: 0x0000004000000004 | 04 00 00 00 40 00 00 00 +x16=0x0000000000000040 is an unknown value +x17=0x0000000b02016290 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff +x18=0x0 is NULL +x19=0x0000000afb041e00 is a thread +x20=0x000000017f06ad50 is pointing into the stack for thread: 0x0000000afb041e00 +x21=0x000000017f06ad20 is pointing into the stack for thread: 0x0000000afb041e00 +x22=0x0000000afbc68018 points into unknown readable memory: 0x00000005016d06b0 | b0 06 6d 01 05 00 00 00 +x23=0x00000005016d06b0 is an oop: org.apache.comet.CometHandleBatchIterator +{0x00000005016d06b0} - klass: 'org/apache/comet/CometHandleBatchIterator' + - ---- fields (total size 2 words): + - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x00000005016c6088} (a02d8c11) +x24=0x0000000000000001 is an unknown value +x25=0x0 is NULL +x26=0x0000000b02033700 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +x27=0x0 is NULL +x28=0x0000000000000001 is an unknown value + + +Top of Stack: (sp=0x000000017f06ac00) +0x000000017f06ac00: 000000017f0652c0 0000000000000001 +0x000000017f06ac10: 0000000b0232d7c0 0000000151212428 +0x000000017f06ac20: 0000000afcc29230 0000000af5652d00 +0x000000017f06ac30: 000000017f06ac67 0000000000000000 +0x000000017f06ac40: 000000017f06ac80 000000014e7800d0 +0x000000017f06ac50: 0000000000000081 0000000af6e0e8d8 +0x000000017f06ac60: 0000000b0232d9a0 0000000af6e0e8d8 +0x000000017f06ac70: 0000000af6e0e8d8 00ffffffffffffff +0x000000017f06ac80: 000000017f06ad10 000000014e7aff58 +0x000000017f06ac90: 0000000000000063 000000017f06ad20 +0x000000017f06aca0: 000000017f06ad10 0000000108231f24 +0x000000017f06acb0: ba714bd2f1340072 0000000afb041e00 +0x000000017f06acc0: 0000000afb041e00 0000000000000000 +0x000000017f06acd0: 0000000136fa8ab0 0000000000000000 +0x000000017f06ace0: 0000000afb0420b0 0000000000000008 +0x000000017f06acf0: 0000000af6eb1bf8 0000000b069bc1f0 +0x000000017f06ad00: 0000000000000001 0000000afb041e00 +0x000000017f06ad10: 000000017f06adc0 000000010821960c +0x000000017f06ad20: 000000010895a6b8 000000d00045a1d0 +0x000000017f06ad30: 000000010000000a 0000000000000014 +0x000000017f06ad40: 0000000000000000 0000000000000008 +0x000000017f06ad50: 000000000000000a 00000001089c3f90 +0x000000017f06ad60: 0000000afb041e00 0000000000000000 +0x000000017f06ad70: 000000017f06adc0 0000000147e84c58 +0x000000017f06ad80: 0000000136fa8ab0 00000005016d8a98 +0x000000017f06ad90: 000000017f071818 0000000000000061 +0x000000017f06ada0: 0000000000000000 00000001089c3f90 +0x000000017f06adb0: 000000017f071738 000000012426772c +0x000000017f06adc0: 000000017f06c980 0000000147e49560 +0x000000017f06add0: 0000000000000000 0000000000000081 +0x000000017f06ade0: 0000000000000082 0000000b0232d9a0 +0x000000017f06adf0: 0000000000000082 000000ffffffffff + +Instructions: (pc=0x0000000108215530) +0x0000000108215430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 +0x0000000108215440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 +0x0000000108215450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 +0x0000000108215460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 +0x0000000108215470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 +0x0000000108215480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 +0x0000000108215490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 +0x00000001082154a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 +0x00000001082154b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa +0x00000001082154c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 +0x00000001082154d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 +0x00000001082154e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 +0x00000001082154f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a +0x0000000108215500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa +0x0000000108215510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 +0x0000000108215520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa +0x0000000108215530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 +0x0000000108215540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 +0x0000000108215550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 +0x0000000108215560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 +0x0000000108215570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 +0x0000000108215580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 +0x0000000108215590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa +0x00000001082155a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 +0x00000001082155b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 +0x00000001082155c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 +0x00000001082155d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 +0x00000001082155e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 +0x00000001082155f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa +0x0000000108215600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 +0x0000000108215610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 +0x0000000108215620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x000000017f0652c0 is pointing into the stack for thread: 0x0000000afb041e00 +stack at sp + 1 slots: 0x0000000000000001 is an unknown value +stack at sp + 2 slots: 0x0000000b0232d7c0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 +stack at sp + 3 slots: 0x0000000151212428: _ZN5comet7parquet15parquet_support15PARQUET_OPTIONS17h20af737816b13564E+0x28650 in /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib at 0x0000000147dd0000 +stack at sp + 4 slots: 0x0000000afcc29230 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +stack at sp + 5 slots: 0x0000000af5652d00 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 +stack at sp + 6 slots: 0x000000017f06ac67 is pointing into the stack for thread: 0x0000000afb041e00 +stack at sp + 7 slots: 0x0 is NULL + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x0000000b07053280, length=80, elements={ +0x000000010295df90, 0x0000000102971cf0, 0x0000000afcd80000, 0x0000000afcd80c00, +0x0000000afcd81200, 0x0000000afcd81800, 0x0000000afcd81e00, 0x0000000afcd82400, +0x0000000afcd82a00, 0x0000000afcd83000, 0x0000000afcbcc600, 0x0000000afbceaa00, +0x0000000af9845e00, 0x0000000af9847000, 0x0000000af9847600, 0x0000000afb9c8000, +0x0000000afb9c8600, 0x0000000afb9c8c00, 0x0000000afb9cb000, 0x0000000afb9cb600, +0x0000000af9944000, 0x0000000af9944600, 0x0000000af9944c00, 0x0000000af9945200, +0x0000000af9945800, 0x0000000af9945e00, 0x0000000af9974600, 0x0000000af9976400, +0x0000000af9976a00, 0x0000000af9977600, 0x0000000af9801200, 0x0000000af9ebb000, +0x0000000afcbcde00, 0x0000000afcbcd800, 0x0000000afcd83600, 0x0000000af99d8000, +0x0000000af99d9800, 0x0000000af99d9e00, 0x0000000af99daa00, 0x0000000af99ec000, +0x0000000af99ec600, 0x0000000af99ecc00, 0x0000000af99ed200, 0x0000000af9a89800, +0x0000000af9a8b600, 0x0000000af9ad0600, 0x0000000af9bb8000, 0x0000000af9145800, +0x0000000afb041200, 0x0000000afb041800, 0x0000000afb041e00, 0x0000000afb042400, +0x0000000afb042a00, 0x0000000afb24ea00, 0x0000000af8168000, 0x0000000af8168600, +0x0000000af8169200, 0x0000000afb503600, 0x0000000af816a400, 0x0000000afb5b8000, +0x0000000af943d800, 0x0000000af6862a00, 0x0000000af6863000, 0x0000000af6863600, +0x0000000af99d9200, 0x0000000afbfe4c00, 0x0000000afb24c000, 0x0000000af6861e00, +0x0000000af6e1c000, 0x0000000af6e1c600, 0x0000000af6e1cc00, 0x0000000af68a7600, +0x0000000af8169800, 0x0000000af816aa00, 0x0000000afb470c00, 0x0000000afb471200, +0x0000000af6860c00, 0x0000000af5631e00, 0x0000000af601c600, 0x0000000af601cc00 +} + +Java Threads: ( => current thread ) + 0x000000010295df90 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4099, stack(0x000000016dd10000,0x000000016e113000)] + 0x0000000102971cf0 JavaThread "Reference Handler" daemon [_thread_blocked, id=32003, stack(0x000000016ed64000,0x000000016f167000)] + 0x0000000afcd80000 JavaThread "Finalizer" daemon [_thread_blocked, id=23811, stack(0x000000016f170000,0x000000016f573000)] + 0x0000000afcd80c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30211, stack(0x000000016f694000,0x000000016fa97000)] + 0x0000000afcd81200 JavaThread "Service Thread" daemon [_thread_blocked, id=26115, stack(0x000000016faa0000,0x000000016fea3000)] + 0x0000000afcd81800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=29955, stack(0x000000016feac000,0x00000001702af000)] + 0x0000000afcd81e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26883, stack(0x00000001702b8000,0x00000001704bb000)] + 0x0000000afcd82400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27139, stack(0x00000001704c4000,0x00000001706c7000)] + 0x0000000afcd82a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=29443, stack(0x00000001706d0000,0x0000000170ad3000)] + 0x0000000afcd83000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=29187, stack(0x0000000170adc000,0x0000000170edf000)] + 0x0000000afcbcc600 JavaThread "Notification Thread" daemon [_thread_blocked, id=28163, stack(0x0000000170ee8000,0x00000001712eb000)] + 0x0000000afbceaa00 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=37635, stack(0x0000000173be4000,0x0000000173fe7000)] + 0x0000000af9845e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=45827, stack(0x0000000175050000,0x0000000175453000)] + 0x0000000af9847000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=46339, stack(0x000000017545c000,0x000000017585f000)] + 0x0000000af9847600 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=63747, stack(0x0000000175868000,0x0000000175c6b000)] + 0x0000000afb9c8000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=46851, stack(0x0000000175c74000,0x0000000176077000)] + 0x0000000afb9c8600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=63491, stack(0x0000000176080000,0x0000000176483000)] + 0x0000000afb9c8c00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=62979, stack(0x000000017648c000,0x000000017688f000)] + 0x0000000afb9cb000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=62467, stack(0x0000000176898000,0x0000000176c9b000)] + 0x0000000afb9cb600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=47363, stack(0x0000000176ca4000,0x00000001770a7000)] + 0x0000000af9944000 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=47875, stack(0x00000001770b0000,0x00000001774b3000)] + 0x0000000af9944600 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=48387, stack(0x00000001774bc000,0x00000001778bf000)] + 0x0000000af9944c00 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=61699, stack(0x00000001778c8000,0x0000000177ccb000)] + 0x0000000af9945200 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=48643, stack(0x0000000177cd4000,0x00000001780d7000)] + 0x0000000af9945800 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=61187, stack(0x00000001780e0000,0x00000001784e3000)] + 0x0000000af9945e00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=49411, stack(0x00000001784ec000,0x00000001788ef000)] + 0x0000000af9974600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=60675, stack(0x00000001788f8000,0x0000000178cfb000)] + 0x0000000af9976400 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=60163, stack(0x0000000178d04000,0x0000000179107000)] + 0x0000000af9976a00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=59907, stack(0x0000000179110000,0x0000000179513000)] + 0x0000000af9977600 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=59395, stack(0x000000017951c000,0x000000017991f000)] + 0x0000000af9801200 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=49923, stack(0x0000000179928000,0x0000000179d2b000)] + 0x0000000af9ebb000 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=58627, stack(0x0000000179d34000,0x000000017a137000)] + 0x0000000afcbcde00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=58115, stack(0x000000017a140000,0x000000017a543000)] + 0x0000000afcbcd800 JavaThread "task-abort-timer" daemon [_thread_blocked, id=50179, stack(0x000000017a54c000,0x000000017a94f000)] + 0x0000000afcd83600 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=50691, stack(0x000000017a958000,0x000000017ad5b000)] + 0x0000000af99d8000 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=57347, stack(0x000000017ad64000,0x000000017b167000)] + 0x0000000af99d9800 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=51203, stack(0x000000017b170000,0x000000017b573000)] + 0x0000000af99d9e00 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=57091, stack(0x000000017b57c000,0x000000017b97f000)] + 0x0000000af99daa00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=56835, stack(0x000000017b988000,0x000000017bd8b000)] + 0x0000000af99ec000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=52227, stack(0x000000017bd94000,0x000000017c197000)] + 0x0000000af99ec600 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=52739, stack(0x000000017c1a0000,0x000000017c5a3000)] + 0x0000000af99ecc00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=56323, stack(0x000000017c5ac000,0x000000017c9af000)] + 0x0000000af99ed200 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=52995, stack(0x000000017c9b8000,0x000000017cdbb000)] + 0x0000000af9a89800 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=55811, stack(0x000000017cdc4000,0x000000017d1c7000)] + 0x0000000af9a8b600 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=55299, stack(0x000000017d1d0000,0x000000017d5d3000)] + 0x0000000af9ad0600 JavaThread "Timer-0" [_thread_blocked, id=54787, stack(0x000000017d5dc000,0x000000017d9df000)] + 0x0000000af9bb8000 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=54531, stack(0x000000017d9e8000,0x000000017ddeb000)] + 0x0000000af9145800 JavaThread "process reaper" daemon [_thread_blocked, id=54275, stack(0x000000017ddf4000,0x000000017de2b000)] + 0x0000000afb041200 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_native, id=86535, stack(0x000000017e458000,0x000000017e85b000)] + 0x0000000afb041800 JavaThread "Executor task launch worker for task 31.0 in stage 4.0 (TID 42)" daemon [_thread_in_vm, id=86031, stack(0x000000017e864000,0x000000017ec67000)] +=>0x0000000afb041e00 JavaThread "Executor task launch worker for task 29.0 in stage 4.0 (TID 40)" daemon [_thread_in_vm, id=66563, stack(0x000000017ec70000,0x000000017f073000)] + 0x0000000afb042400 JavaThread "Executor task launch worker for task 30.0 in stage 4.0 (TID 41)" daemon [_thread_in_vm, id=85507, stack(0x000000017f07c000,0x000000017f47f000)] + 0x0000000afb042a00 JavaThread "Executor task launch worker for task 32.0 in stage 4.0 (TID 43)" daemon [_thread_in_vm, id=84995, stack(0x000000017f488000,0x000000017f88b000)] + 0x0000000afb24ea00 JavaThread "process reaper" daemon [_thread_blocked, id=83971, stack(0x000000016f608000,0x000000016f63f000)] + 0x0000000af8168000 JavaThread "process reaper" daemon [_thread_blocked, id=83715, stack(0x000000016f648000,0x000000016f67f000)] + 0x0000000af8168600 JavaThread "process reaper" daemon [_thread_blocked, id=68867, stack(0x000000017feb8000,0x000000017feef000)] + 0x0000000af8169200 JavaThread "process reaper" daemon [_thread_blocked, id=69123, stack(0x000000017fef8000,0x000000017ff2f000)] + 0x0000000afb503600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=45583, stack(0x000000017de34000,0x000000017e237000)] + 0x0000000af816a400 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=65571, stack(0x000000017faa0000,0x000000017fea3000)] + 0x0000000afb5b8000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=87051, stack(0x0000000300004000,0x0000000300407000)] + 0x0000000af943d800 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=67591, stack(0x0000000300410000,0x0000000300813000)] + 0x0000000af6862a00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69427, stack(0x0000000300e40000,0x0000000301243000)] + 0x0000000af6863000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=70147, stack(0x0000000301658000,0x0000000301a5b000)] + 0x0000000af6863600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=70659, stack(0x0000000301a64000,0x0000000301e67000)] + 0x0000000af99d9200 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81411, stack(0x0000000301e70000,0x0000000302273000)] + 0x0000000afbfe4c00 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=71171, stack(0x000000030227c000,0x000000030267f000)] + 0x0000000afb24c000 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=82243, stack(0x000000030124c000,0x000000030164f000)] + 0x0000000af6861e00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=81167, stack(0x0000000302688000,0x0000000302a8b000)] + 0x0000000af6e1c000 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=80643, stack(0x0000000302a94000,0x0000000302e97000)] + 0x0000000af6e1c600 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=80387, stack(0x0000000302ea0000,0x00000003032a3000)] + 0x0000000af6e1cc00 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=80131, stack(0x00000003032ac000,0x00000003036af000)] + 0x0000000af68a7600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=73491, stack(0x0000000173ff0000,0x00000001741f3000)] + 0x0000000af8169800 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=73999, stack(0x00000001741fc000,0x00000001743ff000)] + 0x0000000af816aa00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=78099, stack(0x0000000174c38000,0x0000000174e3b000)] + 0x0000000afb470c00 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=37403, stack(0x0000000174e44000,0x0000000175047000)] + 0x0000000afb471200 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=45379, stack(0x000000017e240000,0x000000017e443000)] + 0x0000000af6860c00 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=25155, stack(0x000000030081c000,0x0000000300c1f000)] + 0x0000000af5631e00 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=45091, stack(0x000000017f894000,0x000000017fa97000)] + 0x0000000af601c600 JavaThread "C2 CompilerThread4" daemon [_thread_blocked, id=67867, stack(0x0000000300c28000,0x0000000300e2b000)] + 0x0000000af601cc00 JavaThread "C2 CompilerThread5" daemon [_thread_blocked, id=69987, stack(0x00000003040f4000,0x00000003042f7000)] + +Other Threads: + 0x0000000102971070 VMThread "VM Thread" [stack: 0x000000016eb58000,0x000000016ed5b000] [id=20483] + 0x0000000afcb30300 WatcherThread [stack: 0x00000001712f4000,0x00000001714f7000] [id=28675] + 0x0000000102969bf0 GCTaskThread "GC Thread#0" [stack: 0x000000016e11c000,0x000000016e31f000] [id=12803] + 0x0000000afc3f0c00 GCTaskThread "GC Thread#1" [stack: 0x0000000171500000,0x0000000171703000] [id=32771] + 0x0000000afc3f0f00 GCTaskThread "GC Thread#2" [stack: 0x000000017170c000,0x000000017190f000] [id=43267] + 0x0000000afc3f1200 GCTaskThread "GC Thread#3" [stack: 0x0000000171918000,0x0000000171b1b000] [id=42755] + 0x0000000afc3f1500 GCTaskThread "GC Thread#4" [stack: 0x0000000171b24000,0x0000000171d27000] [id=42499] + 0x0000000afc3f1800 GCTaskThread "GC Thread#5" [stack: 0x0000000171d30000,0x0000000171f33000] [id=41987] + 0x0000000afc3f1b00 GCTaskThread "GC Thread#6" [stack: 0x0000000171f3c000,0x000000017213f000] [id=33539] + 0x0000000afc3f1e00 GCTaskThread "GC Thread#7" [stack: 0x0000000172148000,0x000000017234b000] [id=41475] + 0x0000000afc3f2100 GCTaskThread "GC Thread#8" [stack: 0x0000000172354000,0x0000000172557000] [id=41219] + 0x0000000afc3f2400 GCTaskThread "GC Thread#9" [stack: 0x0000000172560000,0x0000000172763000] [id=40963] + 0x0000000afc3f2700 GCTaskThread "GC Thread#10" [stack: 0x000000017276c000,0x000000017296f000] [id=34563] + 0x0000000afc3f2a00 GCTaskThread "GC Thread#11" [stack: 0x0000000172978000,0x0000000172b7b000] [id=34819] + 0x0000000afc3f2d00 GCTaskThread "GC Thread#12" [stack: 0x0000000172b84000,0x0000000172d87000] [id=35075] + 0x0000000afc3f3000 GCTaskThread "GC Thread#13" [stack: 0x0000000172d90000,0x0000000172f93000] [id=39939] + 0x0000000afc3f3300 GCTaskThread "GC Thread#14" [stack: 0x0000000172f9c000,0x000000017319f000] [id=39683] + 0x0000000afc3f3600 GCTaskThread "GC Thread#15" [stack: 0x00000001731a8000,0x00000001733ab000] [id=35843] + 0x0000000afc3f3900 GCTaskThread "GC Thread#16" [stack: 0x00000001733b4000,0x00000001735b7000] [id=36099] + 0x0000000afc3f3c00 GCTaskThread "GC Thread#17" [stack: 0x00000001735c0000,0x00000001737c3000] [id=38915] + 0x0000000afbcc4000 GCTaskThread "GC Thread#18" [stack: 0x00000001737cc000,0x00000001739cf000] [id=36611] + 0x0000000afbcc4300 GCTaskThread "GC Thread#19" [stack: 0x00000001739d8000,0x0000000173bdb000] [id=38147] + 0x000000010296a410 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016e328000,0x000000016e52b000] [id=13059] + 0x000000010296ad10 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016e534000,0x000000016e737000] [id=13827] + 0x0000000afbff7c00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174408000,0x000000017460b000] [id=65283] + 0x0000000afb9b4000 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000174614000,0x0000000174817000] [id=44291] + 0x0000000afb9b4300 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000174820000,0x0000000174a23000] [id=64771] + 0x0000000afb9b4600 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000174a2c000,0x0000000174c2f000] [id=44803] + 0x000000010296cc70 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016e740000,0x000000016e943000] [id=16643] + 0x000000010296d370 ConcurrentGCThread "G1 Service" [stack: 0x000000016e94c000,0x000000016eb4f000] [id=21507] + +Threads with active compile tasks: + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x00000004d5000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x000000d000000000-0x000000d000bc0000-0x000000d000bc0000), size 12320768, SharedBaseAddress: 0x000000d000000000, ArchiveRelocationMode: 1. +Compressed class space mapped at: 0x000000d001000000-0x000000d041000000, reserved size: 1073741824 +Narrow klass base: 0x000000d000000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 28 total, 28 available + Memory: 98304M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 1536M + Heap Max Capacity: 4G + Pre-touch: Disabled + Parallel Workers: 20 + Concurrent Workers: 5 + Concurrent Refinement Workers: 20 + Periodic GC: Disabled + +Heap: + garbage-first heap total 958464K, used 429201K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 115 young (235520K), 2 survivors (4096K) + Metaspace used 151232K, committed 152320K, reserved 1245184K + class space used 18131K, committed 18624K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x00000004d5000000, 0x00000004d5200000, 0x00000004d5200000|100%|HS| |TAMS 0x00000004d5200000, 0x00000004d5000000| Complete +| 1|0x00000004d5200000, 0x00000004d5400000, 0x00000004d5400000|100%|HS| |TAMS 0x00000004d5400000, 0x00000004d5200000| Complete +| 2|0x00000004d5400000, 0x00000004d5600000, 0x00000004d5600000|100%|HC| |TAMS 0x00000004d5600000, 0x00000004d5400000| Complete +| 3|0x00000004d5600000, 0x00000004d5800000, 0x00000004d5800000|100%|HS| |TAMS 0x00000004d5800000, 0x00000004d5600000| Complete +| 4|0x00000004d5800000, 0x00000004d5a00000, 0x00000004d5a00000|100%|HC| |TAMS 0x00000004d5a00000, 0x00000004d5800000| Complete +| 5|0x00000004d5a00000, 0x00000004d5c00000, 0x00000004d5c00000|100%|HC| |TAMS 0x00000004d5c00000, 0x00000004d5a00000| Complete +| 6|0x00000004d5c00000, 0x00000004d5e00000, 0x00000004d5e00000|100%|HC| |TAMS 0x00000004d5e00000, 0x00000004d5c00000| Complete +| 7|0x00000004d5e00000, 0x00000004d6000000, 0x00000004d6000000|100%|HC| |TAMS 0x00000004d6000000, 0x00000004d5e00000| Complete +| 8|0x00000004d6000000, 0x00000004d6200000, 0x00000004d6200000|100%|HC| |TAMS 0x00000004d6200000, 0x00000004d6000000| Complete +| 9|0x00000004d6200000, 0x00000004d6400000, 0x00000004d6400000|100%|HC| |TAMS 0x00000004d6400000, 0x00000004d6200000| Complete +| 10|0x00000004d6400000, 0x00000004d6600000, 0x00000004d6600000|100%|HC| |TAMS 0x00000004d6600000, 0x00000004d6400000| Complete +| 11|0x00000004d6600000, 0x00000004d6800000, 0x00000004d6800000|100%|HS| |TAMS 0x00000004d6800000, 0x00000004d6600000| Complete +| 12|0x00000004d6800000, 0x00000004d6a00000, 0x00000004d6a00000|100%|HS| |TAMS 0x00000004d6a00000, 0x00000004d6800000| Complete +| 13|0x00000004d6a00000, 0x00000004d6c00000, 0x00000004d6c00000|100%|HC| |TAMS 0x00000004d6c00000, 0x00000004d6a00000| Complete +| 14|0x00000004d6c00000, 0x00000004d6e00000, 0x00000004d6e00000|100%|HS| |TAMS 0x00000004d6e00000, 0x00000004d6c00000| Complete +| 15|0x00000004d6e00000, 0x00000004d7000000, 0x00000004d7000000|100%|HC| |TAMS 0x00000004d7000000, 0x00000004d6e00000| Complete +| 16|0x00000004d7000000, 0x00000004d7200000, 0x00000004d7200000|100%|HS| |TAMS 0x00000004d7200000, 0x00000004d7000000| Complete +| 17|0x00000004d7200000, 0x00000004d7400000, 0x00000004d7400000|100%|HS| |TAMS 0x00000004d7400000, 0x00000004d7200000| Complete +| 18|0x00000004d7400000, 0x00000004d7600000, 0x00000004d7600000|100%|HC| |TAMS 0x00000004d7600000, 0x00000004d7400000| Complete +| 19|0x00000004d7600000, 0x00000004d7800000, 0x00000004d7800000|100%| O| |TAMS 0x00000004d7800000, 0x00000004d7600000| Untracked +| 20|0x00000004d7800000, 0x00000004d7a00000, 0x00000004d7a00000|100%| O| |TAMS 0x00000004d7a00000, 0x00000004d7800000| Untracked +| 21|0x00000004d7a00000, 0x00000004d7c00000, 0x00000004d7c00000|100%| O| |TAMS 0x00000004d7c00000, 0x00000004d7a00000| Untracked +| 22|0x00000004d7c00000, 0x00000004d7e00000, 0x00000004d7e00000|100%| O| |TAMS 0x00000004d7e00000, 0x00000004d7c00000| Untracked +| 23|0x00000004d7e00000, 0x00000004d8000000, 0x00000004d8000000|100%| O| |TAMS 0x00000004d8000000, 0x00000004d7e00000| Untracked +| 24|0x00000004d8000000, 0x00000004d8200000, 0x00000004d8200000|100%| O| |TAMS 0x00000004d8200000, 0x00000004d8000000| Untracked +| 25|0x00000004d8200000, 0x00000004d8400000, 0x00000004d8400000|100%| O| |TAMS 0x00000004d8400000, 0x00000004d8200000| Untracked +| 26|0x00000004d8400000, 0x00000004d8600000, 0x00000004d8600000|100%| O| |TAMS 0x00000004d8600000, 0x00000004d8400000| Untracked +| 27|0x00000004d8600000, 0x00000004d8800000, 0x00000004d8800000|100%| O| |TAMS 0x00000004d8800000, 0x00000004d8600000| Untracked +| 28|0x00000004d8800000, 0x00000004d8a00000, 0x00000004d8a00000|100%| O| |TAMS 0x00000004d8a00000, 0x00000004d8800000| Untracked +| 29|0x00000004d8a00000, 0x00000004d8c00000, 0x00000004d8c00000|100%| O| |TAMS 0x00000004d8c00000, 0x00000004d8a00000| Untracked +| 30|0x00000004d8c00000, 0x00000004d8e00000, 0x00000004d8e00000|100%| O| |TAMS 0x00000004d8e00000, 0x00000004d8c00000| Untracked +| 31|0x00000004d8e00000, 0x00000004d9000000, 0x00000004d9000000|100%| O| |TAMS 0x00000004d9000000, 0x00000004d8e00000| Untracked +| 32|0x00000004d9000000, 0x00000004d9200000, 0x00000004d9200000|100%| O| |TAMS 0x00000004d9200000, 0x00000004d9000000| Untracked +| 33|0x00000004d9200000, 0x00000004d9400000, 0x00000004d9400000|100%| O| |TAMS 0x00000004d9400000, 0x00000004d9200000| Untracked +| 34|0x00000004d9400000, 0x00000004d9600000, 0x00000004d9600000|100%| O| |TAMS 0x00000004d9600000, 0x00000004d9400000| Untracked +| 35|0x00000004d9600000, 0x00000004d9800000, 0x00000004d9800000|100%| O| |TAMS 0x00000004d9800000, 0x00000004d9600000| Untracked +| 36|0x00000004d9800000, 0x00000004d9a00000, 0x00000004d9a00000|100%| O| |TAMS 0x00000004d9a00000, 0x00000004d9800000| Untracked +| 37|0x00000004d9a00000, 0x00000004d9c00000, 0x00000004d9c00000|100%| O| |TAMS 0x00000004d9c00000, 0x00000004d9a00000| Untracked +| 38|0x00000004d9c00000, 0x00000004d9e00000, 0x00000004d9e00000|100%| O| |TAMS 0x00000004d9e00000, 0x00000004d9c00000| Untracked +| 39|0x00000004d9e00000, 0x00000004da000000, 0x00000004da000000|100%| O| |TAMS 0x00000004da000000, 0x00000004d9e00000| Untracked +| 40|0x00000004da000000, 0x00000004da200000, 0x00000004da200000|100%| O| |TAMS 0x00000004da200000, 0x00000004da000000| Untracked +| 41|0x00000004da200000, 0x00000004da400000, 0x00000004da400000|100%| O| |TAMS 0x00000004da400000, 0x00000004da200000| Untracked +| 42|0x00000004da400000, 0x00000004da600000, 0x00000004da600000|100%| O| |TAMS 0x00000004da400000, 0x00000004da400000| Untracked +| 43|0x00000004da600000, 0x00000004da800000, 0x00000004da800000|100%| O| |TAMS 0x00000004da800000, 0x00000004da600000| Untracked +| 44|0x00000004da800000, 0x00000004daa00000, 0x00000004daa00000|100%| O| |TAMS 0x00000004daa00000, 0x00000004da800000| Untracked +| 45|0x00000004daa00000, 0x00000004dac00000, 0x00000004dac00000|100%| O| |TAMS 0x00000004dac00000, 0x00000004daa00000| Untracked +| 46|0x00000004dac00000, 0x00000004dae00000, 0x00000004dae00000|100%| O| |TAMS 0x00000004dae00000, 0x00000004dac00000| Untracked +| 47|0x00000004dae00000, 0x00000004db000000, 0x00000004db000000|100%| O| |TAMS 0x00000004db000000, 0x00000004dae00000| Untracked +| 48|0x00000004db000000, 0x00000004db200000, 0x00000004db200000|100%| O| |TAMS 0x00000004db200000, 0x00000004db000000| Untracked +| 49|0x00000004db200000, 0x00000004db2b4e00, 0x00000004db400000| 35%| O| |TAMS 0x00000004db200000, 0x00000004db200000| Untracked +| 50|0x00000004db400000, 0x00000004db600000, 0x00000004db600000|100%|HS| |TAMS 0x00000004db400000, 0x00000004db400000| Complete +| 51|0x00000004db600000, 0x00000004db800000, 0x00000004db800000|100%|HS| |TAMS 0x00000004db600000, 0x00000004db600000| Complete +| 52|0x00000004db800000, 0x00000004dba00000, 0x00000004dba00000|100%|HS| |TAMS 0x00000004db800000, 0x00000004db800000| Complete +| 53|0x00000004dba00000, 0x00000004dbc00000, 0x00000004dbc00000|100%| O| |TAMS 0x00000004dbc00000, 0x00000004dba00000| Untracked +| 54|0x00000004dbc00000, 0x00000004dbe00000, 0x00000004dbe00000|100%| O| |TAMS 0x00000004dbe00000, 0x00000004dbc00000| Untracked +| 55|0x00000004dbe00000, 0x00000004dc000000, 0x00000004dc000000|100%| O| |TAMS 0x00000004dc000000, 0x00000004dbe00000| Untracked +| 56|0x00000004dc000000, 0x00000004dc200000, 0x00000004dc200000|100%| O| |TAMS 0x00000004dc200000, 0x00000004dc000000| Untracked +| 57|0x00000004dc200000, 0x00000004dc400000, 0x00000004dc400000|100%| O| |TAMS 0x00000004dc400000, 0x00000004dc200000| Untracked +| 58|0x00000004dc400000, 0x00000004dc600000, 0x00000004dc600000|100%|HS| |TAMS 0x00000004dc400000, 0x00000004dc400000| Complete +| 59|0x00000004dc600000, 0x00000004dc800000, 0x00000004dc800000|100%|HS| |TAMS 0x00000004dc600000, 0x00000004dc600000| Complete +| 60|0x00000004dc800000, 0x00000004dca00000, 0x00000004dca00000|100%|HS| |TAMS 0x00000004dc800000, 0x00000004dc800000| Complete +| 61|0x00000004dca00000, 0x00000004dcc00000, 0x00000004dcc00000|100%|HC| |TAMS 0x00000004dca00000, 0x00000004dca00000| Complete +| 62|0x00000004dcc00000, 0x00000004dce00000, 0x00000004dce00000|100%|HC| |TAMS 0x00000004dcc00000, 0x00000004dcc00000| Complete +| 63|0x00000004dce00000, 0x00000004dd000000, 0x00000004dd000000|100%| O| |TAMS 0x00000004dd000000, 0x00000004dce00000| Untracked +| 64|0x00000004dd000000, 0x00000004dd200000, 0x00000004dd200000|100%|HS| |TAMS 0x00000004dd000000, 0x00000004dd000000| Complete +| 65|0x00000004dd200000, 0x00000004dd400000, 0x00000004dd400000|100%| O| |TAMS 0x00000004dd2bb200, 0x00000004dd200000| Untracked +| 66|0x00000004dd400000, 0x00000004dd600000, 0x00000004dd600000|100%|HS| |TAMS 0x00000004dd400000, 0x00000004dd400000| Complete +| 67|0x00000004dd600000, 0x00000004dd800000, 0x00000004dd800000|100%|HS| |TAMS 0x00000004dd600000, 0x00000004dd600000| Complete +| 68|0x00000004dd800000, 0x00000004dda00000, 0x00000004dda00000|100%|HS| |TAMS 0x00000004dd800000, 0x00000004dd800000| Complete +| 69|0x00000004dda00000, 0x00000004ddc00000, 0x00000004ddc00000|100%| O| |TAMS 0x00000004ddc00000, 0x00000004dda00000| Untracked +| 70|0x00000004ddc00000, 0x00000004dde00000, 0x00000004dde00000|100%| O| |TAMS 0x00000004dde00000, 0x00000004ddc00000| Untracked +| 71|0x00000004dde00000, 0x00000004de000000, 0x00000004de000000|100%| O| |TAMS 0x00000004de000000, 0x00000004dde00000| Untracked +| 72|0x00000004de000000, 0x00000004de200000, 0x00000004de200000|100%|HS| |TAMS 0x00000004de000000, 0x00000004de000000| Complete +| 73|0x00000004de200000, 0x00000004de400000, 0x00000004de400000|100%|HS| |TAMS 0x00000004de200000, 0x00000004de200000| Complete +| 74|0x00000004de400000, 0x00000004de600000, 0x00000004de600000|100%|HS| |TAMS 0x00000004de400000, 0x00000004de400000| Complete +| 75|0x00000004de600000, 0x00000004de800000, 0x00000004de800000|100%|HS| |TAMS 0x00000004de600000, 0x00000004de600000| Complete +| 76|0x00000004de800000, 0x00000004dea00000, 0x00000004dea00000|100%|HS| |TAMS 0x00000004de800000, 0x00000004de800000| Complete +| 77|0x00000004dea00000, 0x00000004dec00000, 0x00000004dec00000|100%|HS| |TAMS 0x00000004dea00000, 0x00000004dea00000| Complete +| 78|0x00000004dec00000, 0x00000004dee00000, 0x00000004dee00000|100%|HS| |TAMS 0x00000004dec00000, 0x00000004dec00000| Complete +| 79|0x00000004dee00000, 0x00000004df000000, 0x00000004df000000|100%|HS| |TAMS 0x00000004dee00000, 0x00000004dee00000| Complete +| 80|0x00000004df000000, 0x00000004df200000, 0x00000004df200000|100%|HS| |TAMS 0x00000004df000000, 0x00000004df000000| Complete +| 81|0x00000004df200000, 0x00000004df400000, 0x00000004df400000|100%|HS| |TAMS 0x00000004df200000, 0x00000004df200000| Complete +| 82|0x00000004df400000, 0x00000004df600000, 0x00000004df600000|100%|HS| |TAMS 0x00000004df400000, 0x00000004df400000| Complete +| 83|0x00000004df600000, 0x00000004df800000, 0x00000004df800000|100%|HS| |TAMS 0x00000004df600000, 0x00000004df600000| Complete +| 84|0x00000004df800000, 0x00000004dfa00000, 0x00000004dfa00000|100%|HS| |TAMS 0x00000004df800000, 0x00000004df800000| Complete +| 85|0x00000004dfa00000, 0x00000004dfc00000, 0x00000004dfc00000|100%|HS| |TAMS 0x00000004dfa00000, 0x00000004dfa00000| Complete +| 86|0x00000004dfc00000, 0x00000004dfe00000, 0x00000004dfe00000|100%|HS| |TAMS 0x00000004dfc00000, 0x00000004dfc00000| Complete +| 87|0x00000004dfe00000, 0x00000004e0000000, 0x00000004e0000000|100%|HS| |TAMS 0x00000004dfe00000, 0x00000004dfe00000| Complete +| 88|0x00000004e0000000, 0x00000004e0200000, 0x00000004e0200000|100%|HS| |TAMS 0x00000004e0000000, 0x00000004e0000000| Complete +| 89|0x00000004e0200000, 0x00000004e0400000, 0x00000004e0400000|100%|HS| |TAMS 0x00000004e0200000, 0x00000004e0200000| Complete +| 90|0x00000004e0400000, 0x00000004e0600000, 0x00000004e0600000|100%|HS| |TAMS 0x00000004e0400000, 0x00000004e0400000| Complete +| 91|0x00000004e0600000, 0x00000004e0800000, 0x00000004e0800000|100%|HS| |TAMS 0x00000004e0600000, 0x00000004e0600000| Complete +| 92|0x00000004e0800000, 0x00000004e0a00000, 0x00000004e0a00000|100%|HS| |TAMS 0x00000004e0800000, 0x00000004e0800000| Complete +| 93|0x00000004e0a00000, 0x00000004e0c00000, 0x00000004e0c00000|100%|HS| |TAMS 0x00000004e0a00000, 0x00000004e0a00000| Complete +| 94|0x00000004e0c00000, 0x00000004e0e00000, 0x00000004e0e00000|100%|HS| |TAMS 0x00000004e0c00000, 0x00000004e0c00000| Complete +| 95|0x00000004e0e00000, 0x00000004e1000000, 0x00000004e1000000|100%|HS| |TAMS 0x00000004e0e00000, 0x00000004e0e00000| Complete +| 96|0x00000004e1000000, 0x00000004e1000000, 0x00000004e1200000| 0%| F| |TAMS 0x00000004e1000000, 0x00000004e1000000| Untracked +| 97|0x00000004e1200000, 0x00000004e1200000, 0x00000004e1400000| 0%| F| |TAMS 0x00000004e1200000, 0x00000004e1200000| Untracked +| 98|0x00000004e1400000, 0x00000004e1400000, 0x00000004e1600000| 0%| F| |TAMS 0x00000004e1400000, 0x00000004e1400000| Untracked +| 99|0x00000004e1600000, 0x00000004e1600000, 0x00000004e1800000| 0%| F| |TAMS 0x00000004e1600000, 0x00000004e1600000| Untracked +| 100|0x00000004e1800000, 0x00000004e1800000, 0x00000004e1a00000| 0%| F| |TAMS 0x00000004e1800000, 0x00000004e1800000| Untracked +| 101|0x00000004e1a00000, 0x00000004e1a00000, 0x00000004e1c00000| 0%| F| |TAMS 0x00000004e1a00000, 0x00000004e1a00000| Untracked +| 102|0x00000004e1c00000, 0x00000004e1c00000, 0x00000004e1e00000| 0%| F| |TAMS 0x00000004e1c00000, 0x00000004e1c00000| Untracked +| 103|0x00000004e1e00000, 0x00000004e1e00000, 0x00000004e2000000| 0%| F| |TAMS 0x00000004e1e00000, 0x00000004e1e00000| Untracked +| 104|0x00000004e2000000, 0x00000004e2000000, 0x00000004e2200000| 0%| F| |TAMS 0x00000004e2000000, 0x00000004e2000000| Untracked +| 105|0x00000004e2200000, 0x00000004e2200000, 0x00000004e2400000| 0%| F| |TAMS 0x00000004e2200000, 0x00000004e2200000| Untracked +| 106|0x00000004e2400000, 0x00000004e2400000, 0x00000004e2600000| 0%| F| |TAMS 0x00000004e2400000, 0x00000004e2400000| Untracked +| 107|0x00000004e2600000, 0x00000004e2600000, 0x00000004e2800000| 0%| F| |TAMS 0x00000004e2600000, 0x00000004e2600000| Untracked +| 108|0x00000004e2800000, 0x00000004e2800000, 0x00000004e2a00000| 0%| F| |TAMS 0x00000004e2800000, 0x00000004e2800000| Untracked +| 109|0x00000004e2a00000, 0x00000004e2a00000, 0x00000004e2c00000| 0%| F| |TAMS 0x00000004e2a00000, 0x00000004e2a00000| Untracked +| 110|0x00000004e2c00000, 0x00000004e2c00000, 0x00000004e2e00000| 0%| F| |TAMS 0x00000004e2c00000, 0x00000004e2c00000| Untracked +| 111|0x00000004e2e00000, 0x00000004e2e00000, 0x00000004e3000000| 0%| F| |TAMS 0x00000004e2e00000, 0x00000004e2e00000| Untracked +| 112|0x00000004e3000000, 0x00000004e3000000, 0x00000004e3200000| 0%| F| |TAMS 0x00000004e3000000, 0x00000004e3000000| Untracked +| 113|0x00000004e3200000, 0x00000004e3200000, 0x00000004e3400000| 0%| F| |TAMS 0x00000004e3200000, 0x00000004e3200000| Untracked +| 114|0x00000004e3400000, 0x00000004e3400000, 0x00000004e3600000| 0%| F| |TAMS 0x00000004e3400000, 0x00000004e3400000| Untracked +| 115|0x00000004e3600000, 0x00000004e3600000, 0x00000004e3800000| 0%| F| |TAMS 0x00000004e3600000, 0x00000004e3600000| Untracked +| 116|0x00000004e3800000, 0x00000004e3800000, 0x00000004e3a00000| 0%| F| |TAMS 0x00000004e3800000, 0x00000004e3800000| Untracked +| 117|0x00000004e3a00000, 0x00000004e3a00000, 0x00000004e3c00000| 0%| F| |TAMS 0x00000004e3a00000, 0x00000004e3a00000| Untracked +| 118|0x00000004e3c00000, 0x00000004e3c00000, 0x00000004e3e00000| 0%| F| |TAMS 0x00000004e3c00000, 0x00000004e3c00000| Untracked +| 119|0x00000004e3e00000, 0x00000004e3e00000, 0x00000004e4000000| 0%| F| |TAMS 0x00000004e3e00000, 0x00000004e3e00000| Untracked +| 120|0x00000004e4000000, 0x00000004e4000000, 0x00000004e4200000| 0%| F| |TAMS 0x00000004e4000000, 0x00000004e4000000| Untracked +| 121|0x00000004e4200000, 0x00000004e4200000, 0x00000004e4400000| 0%| F| |TAMS 0x00000004e4200000, 0x00000004e4200000| Untracked +| 122|0x00000004e4400000, 0x00000004e4400000, 0x00000004e4600000| 0%| F| |TAMS 0x00000004e4400000, 0x00000004e4400000| Untracked +| 123|0x00000004e4600000, 0x00000004e4600000, 0x00000004e4800000| 0%| F| |TAMS 0x00000004e4600000, 0x00000004e4600000| Untracked +| 124|0x00000004e4800000, 0x00000004e4800000, 0x00000004e4a00000| 0%| F| |TAMS 0x00000004e4800000, 0x00000004e4800000| Untracked +| 125|0x00000004e4a00000, 0x00000004e4a00000, 0x00000004e4c00000| 0%| F| |TAMS 0x00000004e4a00000, 0x00000004e4a00000| Untracked +| 126|0x00000004e4c00000, 0x00000004e4c00000, 0x00000004e4e00000| 0%| F| |TAMS 0x00000004e4c00000, 0x00000004e4c00000| Untracked +| 127|0x00000004e4e00000, 0x00000004e4e00000, 0x00000004e5000000| 0%| F| |TAMS 0x00000004e4e00000, 0x00000004e4e00000| Untracked +| 128|0x00000004e5000000, 0x00000004e5000000, 0x00000004e5200000| 0%| F| |TAMS 0x00000004e5000000, 0x00000004e5000000| Untracked +| 129|0x00000004e5200000, 0x00000004e5200000, 0x00000004e5400000| 0%| F| |TAMS 0x00000004e5200000, 0x00000004e5200000| Untracked +| 130|0x00000004e5400000, 0x00000004e5400000, 0x00000004e5600000| 0%| F| |TAMS 0x00000004e5400000, 0x00000004e5400000| Untracked +| 131|0x00000004e5600000, 0x00000004e5600000, 0x00000004e5800000| 0%| F| |TAMS 0x00000004e5600000, 0x00000004e5600000| Untracked +| 132|0x00000004e5800000, 0x00000004e5800000, 0x00000004e5a00000| 0%| F| |TAMS 0x00000004e5800000, 0x00000004e5800000| Untracked +| 133|0x00000004e5a00000, 0x00000004e5a00000, 0x00000004e5c00000| 0%| F| |TAMS 0x00000004e5a00000, 0x00000004e5a00000| Untracked +| 134|0x00000004e5c00000, 0x00000004e5c00000, 0x00000004e5e00000| 0%| F| |TAMS 0x00000004e5c00000, 0x00000004e5c00000| Untracked +| 135|0x00000004e5e00000, 0x00000004e5e00000, 0x00000004e6000000| 0%| F| |TAMS 0x00000004e5e00000, 0x00000004e5e00000| Untracked +| 136|0x00000004e6000000, 0x00000004e6000000, 0x00000004e6200000| 0%| F| |TAMS 0x00000004e6000000, 0x00000004e6000000| Untracked +| 137|0x00000004e6200000, 0x00000004e6200000, 0x00000004e6400000| 0%| F| |TAMS 0x00000004e6200000, 0x00000004e6200000| Untracked +| 138|0x00000004e6400000, 0x00000004e6400000, 0x00000004e6600000| 0%| F| |TAMS 0x00000004e6400000, 0x00000004e6400000| Untracked +| 139|0x00000004e6600000, 0x00000004e6600000, 0x00000004e6800000| 0%| F| |TAMS 0x00000004e6600000, 0x00000004e6600000| Untracked +| 140|0x00000004e6800000, 0x00000004e6800000, 0x00000004e6a00000| 0%| F| |TAMS 0x00000004e6800000, 0x00000004e6800000| Untracked +| 141|0x00000004e6a00000, 0x00000004e6a00000, 0x00000004e6c00000| 0%| F| |TAMS 0x00000004e6a00000, 0x00000004e6a00000| Untracked +| 142|0x00000004e6c00000, 0x00000004e6c00000, 0x00000004e6e00000| 0%| F| |TAMS 0x00000004e6c00000, 0x00000004e6c00000| Untracked +| 143|0x00000004e6e00000, 0x00000004e6e00000, 0x00000004e7000000| 0%| F| |TAMS 0x00000004e6e00000, 0x00000004e6e00000| Untracked +| 144|0x00000004e7000000, 0x00000004e7000000, 0x00000004e7200000| 0%| F| |TAMS 0x00000004e7000000, 0x00000004e7000000| Untracked +| 145|0x00000004e7200000, 0x00000004e7200000, 0x00000004e7400000| 0%| F| |TAMS 0x00000004e7200000, 0x00000004e7200000| Untracked +| 146|0x00000004e7400000, 0x00000004e7400000, 0x00000004e7600000| 0%| F| |TAMS 0x00000004e7400000, 0x00000004e7400000| Untracked +| 147|0x00000004e7600000, 0x00000004e7600000, 0x00000004e7800000| 0%| F| |TAMS 0x00000004e7600000, 0x00000004e7600000| Untracked +| 148|0x00000004e7800000, 0x00000004e7800000, 0x00000004e7a00000| 0%| F| |TAMS 0x00000004e7800000, 0x00000004e7800000| Untracked +| 149|0x00000004e7a00000, 0x00000004e7a00000, 0x00000004e7c00000| 0%| F| |TAMS 0x00000004e7a00000, 0x00000004e7a00000| Untracked +| 150|0x00000004e7c00000, 0x00000004e7c00000, 0x00000004e7e00000| 0%| F| |TAMS 0x00000004e7c00000, 0x00000004e7c00000| Untracked +| 151|0x00000004e7e00000, 0x00000004e7e00000, 0x00000004e8000000| 0%| F| |TAMS 0x00000004e7e00000, 0x00000004e7e00000| Untracked +| 152|0x00000004e8000000, 0x00000004e8000000, 0x00000004e8200000| 0%| F| |TAMS 0x00000004e8000000, 0x00000004e8000000| Untracked +| 153|0x00000004e8200000, 0x00000004e8200000, 0x00000004e8400000| 0%| F| |TAMS 0x00000004e8200000, 0x00000004e8200000| Untracked +| 154|0x00000004e8400000, 0x00000004e8400000, 0x00000004e8600000| 0%| F| |TAMS 0x00000004e8400000, 0x00000004e8400000| Untracked +| 155|0x00000004e8600000, 0x00000004e8600000, 0x00000004e8800000| 0%| F| |TAMS 0x00000004e8600000, 0x00000004e8600000| Untracked +| 156|0x00000004e8800000, 0x00000004e8800000, 0x00000004e8a00000| 0%| F| |TAMS 0x00000004e8800000, 0x00000004e8800000| Untracked +| 157|0x00000004e8a00000, 0x00000004e8a00000, 0x00000004e8c00000| 0%| F| |TAMS 0x00000004e8a00000, 0x00000004e8a00000| Untracked +| 158|0x00000004e8c00000, 0x00000004e8c00000, 0x00000004e8e00000| 0%| F| |TAMS 0x00000004e8c00000, 0x00000004e8c00000| Untracked +| 159|0x00000004e8e00000, 0x00000004e8e00000, 0x00000004e9000000| 0%| F| |TAMS 0x00000004e8e00000, 0x00000004e8e00000| Untracked +| 160|0x00000004e9000000, 0x00000004e9000000, 0x00000004e9200000| 0%| F| |TAMS 0x00000004e9000000, 0x00000004e9000000| Untracked +| 161|0x00000004e9200000, 0x00000004e9200000, 0x00000004e9400000| 0%| F| |TAMS 0x00000004e9200000, 0x00000004e9200000| Untracked +| 162|0x00000004e9400000, 0x00000004e9400000, 0x00000004e9600000| 0%| F| |TAMS 0x00000004e9400000, 0x00000004e9400000| Untracked +| 163|0x00000004e9600000, 0x00000004e9600000, 0x00000004e9800000| 0%| F| |TAMS 0x00000004e9600000, 0x00000004e9600000| Untracked +| 164|0x00000004e9800000, 0x00000004e9800000, 0x00000004e9a00000| 0%| F| |TAMS 0x00000004e9800000, 0x00000004e9800000| Untracked +| 165|0x00000004e9a00000, 0x00000004e9a00000, 0x00000004e9c00000| 0%| F| |TAMS 0x00000004e9a00000, 0x00000004e9a00000| Untracked +| 166|0x00000004e9c00000, 0x00000004e9c00000, 0x00000004e9e00000| 0%| F| |TAMS 0x00000004e9c00000, 0x00000004e9c00000| Untracked +| 167|0x00000004e9e00000, 0x00000004e9e00000, 0x00000004ea000000| 0%| F| |TAMS 0x00000004e9e00000, 0x00000004e9e00000| Untracked +| 168|0x00000004ea000000, 0x00000004ea000000, 0x00000004ea200000| 0%| F| |TAMS 0x00000004ea000000, 0x00000004ea000000| Untracked +| 169|0x00000004ea200000, 0x00000004ea200000, 0x00000004ea400000| 0%| F| |TAMS 0x00000004ea200000, 0x00000004ea200000| Untracked +| 170|0x00000004ea400000, 0x00000004ea400000, 0x00000004ea600000| 0%| F| |TAMS 0x00000004ea400000, 0x00000004ea400000| Untracked +| 171|0x00000004ea600000, 0x00000004ea600000, 0x00000004ea800000| 0%| F| |TAMS 0x00000004ea600000, 0x00000004ea600000| Untracked +| 172|0x00000004ea800000, 0x00000004ea800000, 0x00000004eaa00000| 0%| F| |TAMS 0x00000004ea800000, 0x00000004ea800000| Untracked +| 173|0x00000004eaa00000, 0x00000004eaa00000, 0x00000004eac00000| 0%| F| |TAMS 0x00000004eaa00000, 0x00000004eaa00000| Untracked +| 174|0x00000004eac00000, 0x00000004eac00000, 0x00000004eae00000| 0%| F| |TAMS 0x00000004eac00000, 0x00000004eac00000| Untracked +| 175|0x00000004eae00000, 0x00000004eae00000, 0x00000004eb000000| 0%| F| |TAMS 0x00000004eae00000, 0x00000004eae00000| Untracked +| 176|0x00000004eb000000, 0x00000004eb000000, 0x00000004eb200000| 0%| F| |TAMS 0x00000004eb000000, 0x00000004eb000000| Untracked +| 177|0x00000004eb200000, 0x00000004eb200000, 0x00000004eb400000| 0%| F| |TAMS 0x00000004eb200000, 0x00000004eb200000| Untracked +| 178|0x00000004eb400000, 0x00000004eb400000, 0x00000004eb600000| 0%| F| |TAMS 0x00000004eb400000, 0x00000004eb400000| Untracked +| 179|0x00000004eb600000, 0x00000004eb600000, 0x00000004eb800000| 0%| F| |TAMS 0x00000004eb600000, 0x00000004eb600000| Untracked +| 180|0x00000004eb800000, 0x00000004eb800000, 0x00000004eba00000| 0%| F| |TAMS 0x00000004eb800000, 0x00000004eb800000| Untracked +| 181|0x00000004eba00000, 0x00000004eba00000, 0x00000004ebc00000| 0%| F| |TAMS 0x00000004eba00000, 0x00000004eba00000| Untracked +| 182|0x00000004ebc00000, 0x00000004ebc00000, 0x00000004ebe00000| 0%| F| |TAMS 0x00000004ebc00000, 0x00000004ebc00000| Untracked +| 183|0x00000004ebe00000, 0x00000004ebe00000, 0x00000004ec000000| 0%| F| |TAMS 0x00000004ebe00000, 0x00000004ebe00000| Untracked +| 184|0x00000004ec000000, 0x00000004ec000000, 0x00000004ec200000| 0%| F| |TAMS 0x00000004ec000000, 0x00000004ec000000| Untracked +| 185|0x00000004ec200000, 0x00000004ec200000, 0x00000004ec400000| 0%| F| |TAMS 0x00000004ec200000, 0x00000004ec200000| Untracked +| 186|0x00000004ec400000, 0x00000004ec487668, 0x00000004ec600000| 26%| S|CS|TAMS 0x00000004ec400000, 0x00000004ec400000| Complete +| 187|0x00000004ec600000, 0x00000004ec800000, 0x00000004ec800000|100%| S|CS|TAMS 0x00000004ec600000, 0x00000004ec600000| Complete +| 188|0x00000004ec800000, 0x00000004ec800000, 0x00000004eca00000| 0%| F| |TAMS 0x00000004ec800000, 0x00000004ec800000| Untracked +| 189|0x00000004eca00000, 0x00000004eca00000, 0x00000004ecc00000| 0%| F| |TAMS 0x00000004eca00000, 0x00000004eca00000| Untracked +| 190|0x00000004ecc00000, 0x00000004ecc00000, 0x00000004ece00000| 0%| F| |TAMS 0x00000004ecc00000, 0x00000004ecc00000| Untracked +| 191|0x00000004ece00000, 0x00000004ece00000, 0x00000004ed000000| 0%| F| |TAMS 0x00000004ece00000, 0x00000004ece00000| Untracked +| 192|0x00000004ed000000, 0x00000004ed000000, 0x00000004ed200000| 0%| F| |TAMS 0x00000004ed000000, 0x00000004ed000000| Untracked +| 193|0x00000004ed200000, 0x00000004ed200000, 0x00000004ed400000| 0%| F| |TAMS 0x00000004ed200000, 0x00000004ed200000| Untracked +| 194|0x00000004ed400000, 0x00000004ed400000, 0x00000004ed600000| 0%| F| |TAMS 0x00000004ed400000, 0x00000004ed400000| Untracked +| 195|0x00000004ed600000, 0x00000004ed600000, 0x00000004ed800000| 0%| F| |TAMS 0x00000004ed600000, 0x00000004ed600000| Untracked +| 196|0x00000004ed800000, 0x00000004ed800000, 0x00000004eda00000| 0%| F| |TAMS 0x00000004ed800000, 0x00000004ed800000| Untracked +| 197|0x00000004eda00000, 0x00000004eda00000, 0x00000004edc00000| 0%| F| |TAMS 0x00000004eda00000, 0x00000004eda00000| Untracked +| 198|0x00000004edc00000, 0x00000004edc00000, 0x00000004ede00000| 0%| F| |TAMS 0x00000004edc00000, 0x00000004edc00000| Untracked +| 199|0x00000004ede00000, 0x00000004ede00000, 0x00000004ee000000| 0%| F| |TAMS 0x00000004ede00000, 0x00000004ede00000| Untracked +| 200|0x00000004ee000000, 0x00000004ee000000, 0x00000004ee200000| 0%| F| |TAMS 0x00000004ee000000, 0x00000004ee000000| Untracked +| 201|0x00000004ee200000, 0x00000004ee200000, 0x00000004ee400000| 0%| F| |TAMS 0x00000004ee200000, 0x00000004ee200000| Untracked +| 202|0x00000004ee400000, 0x00000004ee400000, 0x00000004ee600000| 0%| F| |TAMS 0x00000004ee400000, 0x00000004ee400000| Untracked +| 203|0x00000004ee600000, 0x00000004ee600000, 0x00000004ee800000| 0%| F| |TAMS 0x00000004ee600000, 0x00000004ee600000| Untracked +| 204|0x00000004ee800000, 0x00000004ee800000, 0x00000004eea00000| 0%| F| |TAMS 0x00000004ee800000, 0x00000004ee800000| Untracked +| 205|0x00000004eea00000, 0x00000004eea00000, 0x00000004eec00000| 0%| F| |TAMS 0x00000004eea00000, 0x00000004eea00000| Untracked +| 206|0x00000004eec00000, 0x00000004eec00000, 0x00000004eee00000| 0%| F| |TAMS 0x00000004eec00000, 0x00000004eec00000| Untracked +| 207|0x00000004eee00000, 0x00000004eee00000, 0x00000004ef000000| 0%| F| |TAMS 0x00000004eee00000, 0x00000004eee00000| Untracked +| 208|0x00000004ef000000, 0x00000004ef000000, 0x00000004ef200000| 0%| F| |TAMS 0x00000004ef000000, 0x00000004ef000000| Untracked +| 209|0x00000004ef200000, 0x00000004ef200000, 0x00000004ef400000| 0%| F| |TAMS 0x00000004ef200000, 0x00000004ef200000| Untracked +| 210|0x00000004ef400000, 0x00000004ef400000, 0x00000004ef600000| 0%| F| |TAMS 0x00000004ef400000, 0x00000004ef400000| Untracked +| 211|0x00000004ef600000, 0x00000004ef600000, 0x00000004ef800000| 0%| F| |TAMS 0x00000004ef600000, 0x00000004ef600000| Untracked +| 212|0x00000004ef800000, 0x00000004ef800000, 0x00000004efa00000| 0%| F| |TAMS 0x00000004ef800000, 0x00000004ef800000| Untracked +| 213|0x00000004efa00000, 0x00000004efa00000, 0x00000004efc00000| 0%| F| |TAMS 0x00000004efa00000, 0x00000004efa00000| Untracked +| 214|0x00000004efc00000, 0x00000004efc00000, 0x00000004efe00000| 0%| F| |TAMS 0x00000004efc00000, 0x00000004efc00000| Untracked +| 215|0x00000004efe00000, 0x00000004efe00000, 0x00000004f0000000| 0%| F| |TAMS 0x00000004efe00000, 0x00000004efe00000| Untracked +| 216|0x00000004f0000000, 0x00000004f0000000, 0x00000004f0200000| 0%| F| |TAMS 0x00000004f0000000, 0x00000004f0000000| Untracked +| 217|0x00000004f0200000, 0x00000004f0200000, 0x00000004f0400000| 0%| F| |TAMS 0x00000004f0200000, 0x00000004f0200000| Untracked +| 218|0x00000004f0400000, 0x00000004f0400000, 0x00000004f0600000| 0%| F| |TAMS 0x00000004f0400000, 0x00000004f0400000| Untracked +| 219|0x00000004f0600000, 0x00000004f0600000, 0x00000004f0800000| 0%| F| |TAMS 0x00000004f0600000, 0x00000004f0600000| Untracked +| 220|0x00000004f0800000, 0x00000004f0800000, 0x00000004f0a00000| 0%| F| |TAMS 0x00000004f0800000, 0x00000004f0800000| Untracked +| 221|0x00000004f0a00000, 0x00000004f0a00000, 0x00000004f0c00000| 0%| F| |TAMS 0x00000004f0a00000, 0x00000004f0a00000| Untracked +| 222|0x00000004f0c00000, 0x00000004f0c00000, 0x00000004f0e00000| 0%| F| |TAMS 0x00000004f0c00000, 0x00000004f0c00000| Untracked +| 223|0x00000004f0e00000, 0x00000004f0e00000, 0x00000004f1000000| 0%| F| |TAMS 0x00000004f0e00000, 0x00000004f0e00000| Untracked +| 224|0x00000004f1000000, 0x00000004f1000000, 0x00000004f1200000| 0%| F| |TAMS 0x00000004f1000000, 0x00000004f1000000| Untracked +| 225|0x00000004f1200000, 0x00000004f1200000, 0x00000004f1400000| 0%| F| |TAMS 0x00000004f1200000, 0x00000004f1200000| Untracked +| 226|0x00000004f1400000, 0x00000004f1400000, 0x00000004f1600000| 0%| F| |TAMS 0x00000004f1400000, 0x00000004f1400000| Untracked +| 227|0x00000004f1600000, 0x00000004f1600000, 0x00000004f1800000| 0%| F| |TAMS 0x00000004f1600000, 0x00000004f1600000| Untracked +| 228|0x00000004f1800000, 0x00000004f1800000, 0x00000004f1a00000| 0%| F| |TAMS 0x00000004f1800000, 0x00000004f1800000| Untracked +| 229|0x00000004f1a00000, 0x00000004f1a00000, 0x00000004f1c00000| 0%| F| |TAMS 0x00000004f1a00000, 0x00000004f1a00000| Untracked +| 230|0x00000004f1c00000, 0x00000004f1c00000, 0x00000004f1e00000| 0%| F| |TAMS 0x00000004f1c00000, 0x00000004f1c00000| Untracked +| 231|0x00000004f1e00000, 0x00000004f1e00000, 0x00000004f2000000| 0%| F| |TAMS 0x00000004f1e00000, 0x00000004f1e00000| Untracked +| 232|0x00000004f2000000, 0x00000004f2000000, 0x00000004f2200000| 0%| F| |TAMS 0x00000004f2000000, 0x00000004f2000000| Untracked +| 233|0x00000004f2200000, 0x00000004f2200000, 0x00000004f2400000| 0%| F| |TAMS 0x00000004f2200000, 0x00000004f2200000| Untracked +| 234|0x00000004f2400000, 0x00000004f2400000, 0x00000004f2600000| 0%| F| |TAMS 0x00000004f2400000, 0x00000004f2400000| Untracked +| 235|0x00000004f2600000, 0x00000004f2600000, 0x00000004f2800000| 0%| F| |TAMS 0x00000004f2600000, 0x00000004f2600000| Untracked +| 236|0x00000004f2800000, 0x00000004f2800000, 0x00000004f2a00000| 0%| F| |TAMS 0x00000004f2800000, 0x00000004f2800000| Untracked +| 237|0x00000004f2a00000, 0x00000004f2a00000, 0x00000004f2c00000| 0%| F| |TAMS 0x00000004f2a00000, 0x00000004f2a00000| Untracked +| 238|0x00000004f2c00000, 0x00000004f2c00000, 0x00000004f2e00000| 0%| F| |TAMS 0x00000004f2c00000, 0x00000004f2c00000| Untracked +| 239|0x00000004f2e00000, 0x00000004f2e00000, 0x00000004f3000000| 0%| F| |TAMS 0x00000004f2e00000, 0x00000004f2e00000| Untracked +| 240|0x00000004f3000000, 0x00000004f3000000, 0x00000004f3200000| 0%| F| |TAMS 0x00000004f3000000, 0x00000004f3000000| Untracked +| 241|0x00000004f3200000, 0x00000004f3200000, 0x00000004f3400000| 0%| F| |TAMS 0x00000004f3200000, 0x00000004f3200000| Untracked +| 242|0x00000004f3400000, 0x00000004f3400000, 0x00000004f3600000| 0%| F| |TAMS 0x00000004f3400000, 0x00000004f3400000| Untracked +| 243|0x00000004f3600000, 0x00000004f3600000, 0x00000004f3800000| 0%| F| |TAMS 0x00000004f3600000, 0x00000004f3600000| Untracked +| 244|0x00000004f3800000, 0x00000004f3800000, 0x00000004f3a00000| 0%| F| |TAMS 0x00000004f3800000, 0x00000004f3800000| Untracked +| 245|0x00000004f3a00000, 0x00000004f3a00000, 0x00000004f3c00000| 0%| F| |TAMS 0x00000004f3a00000, 0x00000004f3a00000| Untracked +| 246|0x00000004f3c00000, 0x00000004f3c00000, 0x00000004f3e00000| 0%| F| |TAMS 0x00000004f3c00000, 0x00000004f3c00000| Untracked +| 247|0x00000004f3e00000, 0x00000004f3e00000, 0x00000004f4000000| 0%| F| |TAMS 0x00000004f3e00000, 0x00000004f3e00000| Untracked +| 248|0x00000004f4000000, 0x00000004f4000000, 0x00000004f4200000| 0%| F| |TAMS 0x00000004f4000000, 0x00000004f4000000| Untracked +| 249|0x00000004f4200000, 0x00000004f4200000, 0x00000004f4400000| 0%| F| |TAMS 0x00000004f4200000, 0x00000004f4200000| Untracked +| 250|0x00000004f4400000, 0x00000004f4400000, 0x00000004f4600000| 0%| F| |TAMS 0x00000004f4400000, 0x00000004f4400000| Untracked +| 251|0x00000004f4600000, 0x00000004f4600000, 0x00000004f4800000| 0%| F| |TAMS 0x00000004f4600000, 0x00000004f4600000| Untracked +| 252|0x00000004f4800000, 0x00000004f4800000, 0x00000004f4a00000| 0%| F| |TAMS 0x00000004f4800000, 0x00000004f4800000| Untracked +| 253|0x00000004f4a00000, 0x00000004f4a00000, 0x00000004f4c00000| 0%| F| |TAMS 0x00000004f4a00000, 0x00000004f4a00000| Untracked +| 254|0x00000004f4c00000, 0x00000004f4c00000, 0x00000004f4e00000| 0%| F| |TAMS 0x00000004f4c00000, 0x00000004f4c00000| Untracked +| 255|0x00000004f4e00000, 0x00000004f4e00000, 0x00000004f5000000| 0%| F| |TAMS 0x00000004f4e00000, 0x00000004f4e00000| Untracked +| 256|0x00000004f5000000, 0x00000004f5000000, 0x00000004f5200000| 0%| F| |TAMS 0x00000004f5000000, 0x00000004f5000000| Untracked +| 257|0x00000004f5200000, 0x00000004f5200000, 0x00000004f5400000| 0%| F| |TAMS 0x00000004f5200000, 0x00000004f5200000| Untracked +| 258|0x00000004f5400000, 0x00000004f5400000, 0x00000004f5600000| 0%| F| |TAMS 0x00000004f5400000, 0x00000004f5400000| Untracked +| 259|0x00000004f5600000, 0x00000004f5600000, 0x00000004f5800000| 0%| F| |TAMS 0x00000004f5600000, 0x00000004f5600000| Untracked +| 260|0x00000004f5800000, 0x00000004f5800000, 0x00000004f5a00000| 0%| F| |TAMS 0x00000004f5800000, 0x00000004f5800000| Untracked +| 261|0x00000004f5a00000, 0x00000004f5a00000, 0x00000004f5c00000| 0%| F| |TAMS 0x00000004f5a00000, 0x00000004f5a00000| Untracked +| 262|0x00000004f5c00000, 0x00000004f5c00000, 0x00000004f5e00000| 0%| F| |TAMS 0x00000004f5c00000, 0x00000004f5c00000| Untracked +| 263|0x00000004f5e00000, 0x00000004f5e00000, 0x00000004f6000000| 0%| F| |TAMS 0x00000004f5e00000, 0x00000004f5e00000| Untracked +| 264|0x00000004f6000000, 0x00000004f6000000, 0x00000004f6200000| 0%| F| |TAMS 0x00000004f6000000, 0x00000004f6000000| Untracked +| 265|0x00000004f6200000, 0x00000004f6200000, 0x00000004f6400000| 0%| F| |TAMS 0x00000004f6200000, 0x00000004f6200000| Untracked +| 266|0x00000004f6400000, 0x00000004f6400000, 0x00000004f6600000| 0%| F| |TAMS 0x00000004f6400000, 0x00000004f6400000| Untracked +| 267|0x00000004f6600000, 0x00000004f6600000, 0x00000004f6800000| 0%| F| |TAMS 0x00000004f6600000, 0x00000004f6600000| Untracked +| 268|0x00000004f6800000, 0x00000004f6800000, 0x00000004f6a00000| 0%| F| |TAMS 0x00000004f6800000, 0x00000004f6800000| Untracked +| 269|0x00000004f6a00000, 0x00000004f6a00000, 0x00000004f6c00000| 0%| F| |TAMS 0x00000004f6a00000, 0x00000004f6a00000| Untracked +| 270|0x00000004f6c00000, 0x00000004f6c00000, 0x00000004f6e00000| 0%| F| |TAMS 0x00000004f6c00000, 0x00000004f6c00000| Untracked +| 271|0x00000004f6e00000, 0x00000004f6e00000, 0x00000004f7000000| 0%| F| |TAMS 0x00000004f6e00000, 0x00000004f6e00000| Untracked +| 272|0x00000004f7000000, 0x00000004f7000000, 0x00000004f7200000| 0%| F| |TAMS 0x00000004f7000000, 0x00000004f7000000| Untracked +| 273|0x00000004f7200000, 0x00000004f7200000, 0x00000004f7400000| 0%| F| |TAMS 0x00000004f7200000, 0x00000004f7200000| Untracked +| 274|0x00000004f7400000, 0x00000004f7400000, 0x00000004f7600000| 0%| F| |TAMS 0x00000004f7400000, 0x00000004f7400000| Untracked +| 275|0x00000004f7600000, 0x00000004f7600000, 0x00000004f7800000| 0%| F| |TAMS 0x00000004f7600000, 0x00000004f7600000| Untracked +| 276|0x00000004f7800000, 0x00000004f7800000, 0x00000004f7a00000| 0%| F| |TAMS 0x00000004f7800000, 0x00000004f7800000| Untracked +| 277|0x00000004f7a00000, 0x00000004f7a00000, 0x00000004f7c00000| 0%| F| |TAMS 0x00000004f7a00000, 0x00000004f7a00000| Untracked +| 278|0x00000004f7c00000, 0x00000004f7c00000, 0x00000004f7e00000| 0%| F| |TAMS 0x00000004f7c00000, 0x00000004f7c00000| Untracked +| 279|0x00000004f7e00000, 0x00000004f7e00000, 0x00000004f8000000| 0%| F| |TAMS 0x00000004f7e00000, 0x00000004f7e00000| Untracked +| 280|0x00000004f8000000, 0x00000004f8000000, 0x00000004f8200000| 0%| F| |TAMS 0x00000004f8000000, 0x00000004f8000000| Untracked +| 281|0x00000004f8200000, 0x00000004f8200000, 0x00000004f8400000| 0%| F| |TAMS 0x00000004f8200000, 0x00000004f8200000| Untracked +| 282|0x00000004f8400000, 0x00000004f8400000, 0x00000004f8600000| 0%| F| |TAMS 0x00000004f8400000, 0x00000004f8400000| Untracked +| 283|0x00000004f8600000, 0x00000004f8600000, 0x00000004f8800000| 0%| F| |TAMS 0x00000004f8600000, 0x00000004f8600000| Untracked +| 284|0x00000004f8800000, 0x00000004f8800000, 0x00000004f8a00000| 0%| F| |TAMS 0x00000004f8800000, 0x00000004f8800000| Untracked +| 285|0x00000004f8a00000, 0x00000004f8a00000, 0x00000004f8c00000| 0%| F| |TAMS 0x00000004f8a00000, 0x00000004f8a00000| Untracked +| 286|0x00000004f8c00000, 0x00000004f8c00000, 0x00000004f8e00000| 0%| F| |TAMS 0x00000004f8c00000, 0x00000004f8c00000| Untracked +| 287|0x00000004f8e00000, 0x00000004f8e00000, 0x00000004f9000000| 0%| F| |TAMS 0x00000004f8e00000, 0x00000004f8e00000| Untracked +| 288|0x00000004f9000000, 0x00000004f9000000, 0x00000004f9200000| 0%| F| |TAMS 0x00000004f9000000, 0x00000004f9000000| Untracked +| 289|0x00000004f9200000, 0x00000004f9200000, 0x00000004f9400000| 0%| F| |TAMS 0x00000004f9200000, 0x00000004f9200000| Untracked +| 290|0x00000004f9400000, 0x00000004f9400000, 0x00000004f9600000| 0%| F| |TAMS 0x00000004f9400000, 0x00000004f9400000| Untracked +| 291|0x00000004f9600000, 0x00000004f9600000, 0x00000004f9800000| 0%| F| |TAMS 0x00000004f9600000, 0x00000004f9600000| Untracked +| 292|0x00000004f9800000, 0x00000004f9800000, 0x00000004f9a00000| 0%| F| |TAMS 0x00000004f9800000, 0x00000004f9800000| Untracked +| 293|0x00000004f9a00000, 0x00000004f9a00000, 0x00000004f9c00000| 0%| F| |TAMS 0x00000004f9a00000, 0x00000004f9a00000| Untracked +| 294|0x00000004f9c00000, 0x00000004f9c00000, 0x00000004f9e00000| 0%| F| |TAMS 0x00000004f9c00000, 0x00000004f9c00000| Untracked +| 295|0x00000004f9e00000, 0x00000004f9e00000, 0x00000004fa000000| 0%| F| |TAMS 0x00000004f9e00000, 0x00000004f9e00000| Untracked +| 296|0x00000004fa000000, 0x00000004fa000000, 0x00000004fa200000| 0%| F| |TAMS 0x00000004fa000000, 0x00000004fa000000| Untracked +| 297|0x00000004fa200000, 0x00000004fa200000, 0x00000004fa400000| 0%| F| |TAMS 0x00000004fa200000, 0x00000004fa200000| Untracked +| 298|0x00000004fa400000, 0x00000004fa400000, 0x00000004fa600000| 0%| F| |TAMS 0x00000004fa400000, 0x00000004fa400000| Untracked +| 299|0x00000004fa600000, 0x00000004fa600000, 0x00000004fa800000| 0%| F| |TAMS 0x00000004fa600000, 0x00000004fa600000| Untracked +| 300|0x00000004fa800000, 0x00000004fa800000, 0x00000004faa00000| 0%| F| |TAMS 0x00000004fa800000, 0x00000004fa800000| Untracked +| 301|0x00000004faa00000, 0x00000004faa00000, 0x00000004fac00000| 0%| F| |TAMS 0x00000004faa00000, 0x00000004faa00000| Untracked +| 302|0x00000004fac00000, 0x00000004fac00000, 0x00000004fae00000| 0%| F| |TAMS 0x00000004fac00000, 0x00000004fac00000| Untracked +| 303|0x00000004fae00000, 0x00000004fae00000, 0x00000004fb000000| 0%| F| |TAMS 0x00000004fae00000, 0x00000004fae00000| Untracked +| 304|0x00000004fb000000, 0x00000004fb000000, 0x00000004fb200000| 0%| F| |TAMS 0x00000004fb000000, 0x00000004fb000000| Untracked +| 305|0x00000004fb200000, 0x00000004fb200000, 0x00000004fb400000| 0%| F| |TAMS 0x00000004fb200000, 0x00000004fb200000| Untracked +| 306|0x00000004fb400000, 0x00000004fb400000, 0x00000004fb600000| 0%| F| |TAMS 0x00000004fb400000, 0x00000004fb400000| Untracked +| 307|0x00000004fb600000, 0x00000004fb600000, 0x00000004fb800000| 0%| F| |TAMS 0x00000004fb600000, 0x00000004fb600000| Untracked +| 308|0x00000004fb800000, 0x00000004fb800000, 0x00000004fba00000| 0%| F| |TAMS 0x00000004fb800000, 0x00000004fb800000| Untracked +| 309|0x00000004fba00000, 0x00000004fba00000, 0x00000004fbc00000| 0%| F| |TAMS 0x00000004fba00000, 0x00000004fba00000| Untracked +| 310|0x00000004fbc00000, 0x00000004fbc00000, 0x00000004fbe00000| 0%| F| |TAMS 0x00000004fbc00000, 0x00000004fbc00000| Untracked +| 311|0x00000004fbe00000, 0x00000004fbe00000, 0x00000004fc000000| 0%| F| |TAMS 0x00000004fbe00000, 0x00000004fbe00000| Untracked +| 312|0x00000004fc000000, 0x00000004fc000000, 0x00000004fc200000| 0%| F| |TAMS 0x00000004fc000000, 0x00000004fc000000| Untracked +| 313|0x00000004fc200000, 0x00000004fc200000, 0x00000004fc400000| 0%| F| |TAMS 0x00000004fc200000, 0x00000004fc200000| Untracked +| 314|0x00000004fc400000, 0x00000004fc400000, 0x00000004fc600000| 0%| F| |TAMS 0x00000004fc400000, 0x00000004fc400000| Untracked +| 315|0x00000004fc600000, 0x00000004fc600000, 0x00000004fc800000| 0%| F| |TAMS 0x00000004fc600000, 0x00000004fc600000| Untracked +| 316|0x00000004fc800000, 0x00000004fc800000, 0x00000004fca00000| 0%| F| |TAMS 0x00000004fc800000, 0x00000004fc800000| Untracked +| 317|0x00000004fca00000, 0x00000004fca00000, 0x00000004fcc00000| 0%| F| |TAMS 0x00000004fca00000, 0x00000004fca00000| Untracked +| 318|0x00000004fcc00000, 0x00000004fcc00000, 0x00000004fce00000| 0%| F| |TAMS 0x00000004fcc00000, 0x00000004fcc00000| Untracked +| 319|0x00000004fce00000, 0x00000004fce00000, 0x00000004fd000000| 0%| F| |TAMS 0x00000004fce00000, 0x00000004fce00000| Untracked +| 320|0x00000004fd000000, 0x00000004fd000000, 0x00000004fd200000| 0%| F| |TAMS 0x00000004fd000000, 0x00000004fd000000| Untracked +| 321|0x00000004fd200000, 0x00000004fd200000, 0x00000004fd400000| 0%| F| |TAMS 0x00000004fd200000, 0x00000004fd200000| Untracked +| 322|0x00000004fd400000, 0x00000004fd400000, 0x00000004fd600000| 0%| F| |TAMS 0x00000004fd400000, 0x00000004fd400000| Untracked +| 323|0x00000004fd600000, 0x00000004fd600000, 0x00000004fd800000| 0%| F| |TAMS 0x00000004fd600000, 0x00000004fd600000| Untracked +| 324|0x00000004fd800000, 0x00000004fd800000, 0x00000004fda00000| 0%| F| |TAMS 0x00000004fd800000, 0x00000004fd800000| Untracked +| 325|0x00000004fda00000, 0x00000004fda00000, 0x00000004fdc00000| 0%| F| |TAMS 0x00000004fda00000, 0x00000004fda00000| Untracked +| 326|0x00000004fdc00000, 0x00000004fdc00000, 0x00000004fde00000| 0%| F| |TAMS 0x00000004fdc00000, 0x00000004fdc00000| Untracked +| 327|0x00000004fde00000, 0x00000004fde00000, 0x00000004fe000000| 0%| F| |TAMS 0x00000004fde00000, 0x00000004fde00000| Untracked +| 328|0x00000004fe000000, 0x00000004fe000000, 0x00000004fe200000| 0%| F| |TAMS 0x00000004fe000000, 0x00000004fe000000| Untracked +| 329|0x00000004fe200000, 0x00000004fe200000, 0x00000004fe400000| 0%| F| |TAMS 0x00000004fe200000, 0x00000004fe200000| Untracked +| 330|0x00000004fe400000, 0x00000004fe400000, 0x00000004fe600000| 0%| F| |TAMS 0x00000004fe400000, 0x00000004fe400000| Untracked +| 331|0x00000004fe600000, 0x00000004fe600000, 0x00000004fe800000| 0%| F| |TAMS 0x00000004fe600000, 0x00000004fe600000| Untracked +| 332|0x00000004fe800000, 0x00000004fe800000, 0x00000004fea00000| 0%| F| |TAMS 0x00000004fe800000, 0x00000004fe800000| Untracked +| 333|0x00000004fea00000, 0x00000004fea00000, 0x00000004fec00000| 0%| F| |TAMS 0x00000004fea00000, 0x00000004fea00000| Untracked +| 334|0x00000004fec00000, 0x00000004fec00000, 0x00000004fee00000| 0%| F| |TAMS 0x00000004fec00000, 0x00000004fec00000| Untracked +| 335|0x00000004fee00000, 0x00000004fee00000, 0x00000004ff000000| 0%| F| |TAMS 0x00000004fee00000, 0x00000004fee00000| Untracked +| 336|0x00000004ff000000, 0x00000004ff000000, 0x00000004ff200000| 0%| F| |TAMS 0x00000004ff000000, 0x00000004ff000000| Untracked +| 337|0x00000004ff200000, 0x00000004ff200000, 0x00000004ff400000| 0%| F| |TAMS 0x00000004ff200000, 0x00000004ff200000| Untracked +| 338|0x00000004ff400000, 0x00000004ff400000, 0x00000004ff600000| 0%| F| |TAMS 0x00000004ff400000, 0x00000004ff400000| Untracked +| 339|0x00000004ff600000, 0x00000004ff600000, 0x00000004ff800000| 0%| F| |TAMS 0x00000004ff600000, 0x00000004ff600000| Untracked +| 340|0x00000004ff800000, 0x00000004ff800000, 0x00000004ffa00000| 0%| F| |TAMS 0x00000004ff800000, 0x00000004ff800000| Untracked +| 341|0x00000004ffa00000, 0x00000004ffa00000, 0x00000004ffc00000| 0%| F| |TAMS 0x00000004ffa00000, 0x00000004ffa00000| Untracked +| 342|0x00000004ffc00000, 0x00000004ffc00000, 0x00000004ffe00000| 0%| F| |TAMS 0x00000004ffc00000, 0x00000004ffc00000| Untracked +| 343|0x00000004ffe00000, 0x00000004ffe00000, 0x0000000500000000| 0%| F| |TAMS 0x00000004ffe00000, 0x00000004ffe00000| Untracked +| 344|0x0000000500000000, 0x0000000500000000, 0x0000000500200000| 0%| F| |TAMS 0x0000000500000000, 0x0000000500000000| Untracked +| 345|0x0000000500200000, 0x0000000500200000, 0x0000000500400000| 0%| F| |TAMS 0x0000000500200000, 0x0000000500200000| Untracked +| 346|0x0000000500400000, 0x0000000500400000, 0x0000000500600000| 0%| F| |TAMS 0x0000000500400000, 0x0000000500400000| Untracked +| 347|0x0000000500600000, 0x0000000500600000, 0x0000000500800000| 0%| F| |TAMS 0x0000000500600000, 0x0000000500600000| Untracked +| 348|0x0000000500800000, 0x0000000500800000, 0x0000000500a00000| 0%| F| |TAMS 0x0000000500800000, 0x0000000500800000| Untracked +| 349|0x0000000500a00000, 0x0000000500a00000, 0x0000000500c00000| 0%| F| |TAMS 0x0000000500a00000, 0x0000000500a00000| Untracked +| 350|0x0000000500c00000, 0x0000000500c00000, 0x0000000500e00000| 0%| F| |TAMS 0x0000000500c00000, 0x0000000500c00000| Untracked +| 351|0x0000000500e00000, 0x0000000500e00000, 0x0000000501000000| 0%| F| |TAMS 0x0000000500e00000, 0x0000000500e00000| Untracked +| 352|0x0000000501000000, 0x0000000501000000, 0x0000000501200000| 0%| F| |TAMS 0x0000000501000000, 0x0000000501000000| Untracked +| 353|0x0000000501200000, 0x000000050132f438, 0x0000000501400000| 59%| E| |TAMS 0x0000000501200000, 0x0000000501200000| Complete +| 354|0x0000000501400000, 0x0000000501600000, 0x0000000501600000|100%| E|CS|TAMS 0x0000000501400000, 0x0000000501400000| Complete +| 355|0x0000000501600000, 0x0000000501800000, 0x0000000501800000|100%| E|CS|TAMS 0x0000000501600000, 0x0000000501600000| Complete +| 356|0x0000000501800000, 0x0000000501a00000, 0x0000000501a00000|100%| E|CS|TAMS 0x0000000501800000, 0x0000000501800000| Complete +| 357|0x0000000501a00000, 0x0000000501c00000, 0x0000000501c00000|100%| E|CS|TAMS 0x0000000501a00000, 0x0000000501a00000| Complete +| 358|0x0000000501c00000, 0x0000000501e00000, 0x0000000501e00000|100%| E|CS|TAMS 0x0000000501c00000, 0x0000000501c00000| Complete +| 359|0x0000000501e00000, 0x0000000502000000, 0x0000000502000000|100%| E|CS|TAMS 0x0000000501e00000, 0x0000000501e00000| Complete +| 360|0x0000000502000000, 0x0000000502200000, 0x0000000502200000|100%| E|CS|TAMS 0x0000000502000000, 0x0000000502000000| Complete +| 361|0x0000000502200000, 0x0000000502400000, 0x0000000502400000|100%| E|CS|TAMS 0x0000000502200000, 0x0000000502200000| Complete +| 362|0x0000000502400000, 0x0000000502600000, 0x0000000502600000|100%| E|CS|TAMS 0x0000000502400000, 0x0000000502400000| Complete +| 363|0x0000000502600000, 0x0000000502800000, 0x0000000502800000|100%| E|CS|TAMS 0x0000000502600000, 0x0000000502600000| Complete +| 364|0x0000000502800000, 0x0000000502a00000, 0x0000000502a00000|100%| E|CS|TAMS 0x0000000502800000, 0x0000000502800000| Complete +| 365|0x0000000502a00000, 0x0000000502c00000, 0x0000000502c00000|100%| E|CS|TAMS 0x0000000502a00000, 0x0000000502a00000| Complete +| 366|0x0000000502c00000, 0x0000000502e00000, 0x0000000502e00000|100%| E|CS|TAMS 0x0000000502c00000, 0x0000000502c00000| Complete +| 367|0x0000000502e00000, 0x0000000503000000, 0x0000000503000000|100%| E|CS|TAMS 0x0000000502e00000, 0x0000000502e00000| Complete +| 368|0x0000000503000000, 0x0000000503200000, 0x0000000503200000|100%| E|CS|TAMS 0x0000000503000000, 0x0000000503000000| Complete +| 369|0x0000000503200000, 0x0000000503400000, 0x0000000503400000|100%| E|CS|TAMS 0x0000000503200000, 0x0000000503200000| Complete +| 370|0x0000000503400000, 0x0000000503600000, 0x0000000503600000|100%| E|CS|TAMS 0x0000000503400000, 0x0000000503400000| Complete +| 371|0x0000000503600000, 0x0000000503800000, 0x0000000503800000|100%| E|CS|TAMS 0x0000000503600000, 0x0000000503600000| Complete +| 372|0x0000000503800000, 0x0000000503a00000, 0x0000000503a00000|100%| E|CS|TAMS 0x0000000503800000, 0x0000000503800000| Complete +| 373|0x0000000503a00000, 0x0000000503c00000, 0x0000000503c00000|100%| E|CS|TAMS 0x0000000503a00000, 0x0000000503a00000| Complete +| 374|0x0000000503c00000, 0x0000000503e00000, 0x0000000503e00000|100%| E|CS|TAMS 0x0000000503c00000, 0x0000000503c00000| Complete +| 375|0x0000000503e00000, 0x0000000504000000, 0x0000000504000000|100%| E|CS|TAMS 0x0000000503e00000, 0x0000000503e00000| Complete +| 376|0x0000000504000000, 0x0000000504200000, 0x0000000504200000|100%| E|CS|TAMS 0x0000000504000000, 0x0000000504000000| Complete +| 377|0x0000000504200000, 0x0000000504400000, 0x0000000504400000|100%| E|CS|TAMS 0x0000000504200000, 0x0000000504200000| Complete +| 378|0x0000000504400000, 0x0000000504600000, 0x0000000504600000|100%| E|CS|TAMS 0x0000000504400000, 0x0000000504400000| Complete +| 379|0x0000000504600000, 0x0000000504800000, 0x0000000504800000|100%| E|CS|TAMS 0x0000000504600000, 0x0000000504600000| Complete +| 380|0x0000000504800000, 0x0000000504a00000, 0x0000000504a00000|100%| E|CS|TAMS 0x0000000504800000, 0x0000000504800000| Complete +| 381|0x0000000504a00000, 0x0000000504c00000, 0x0000000504c00000|100%| E|CS|TAMS 0x0000000504a00000, 0x0000000504a00000| Complete +| 382|0x0000000504c00000, 0x0000000504e00000, 0x0000000504e00000|100%| E|CS|TAMS 0x0000000504c00000, 0x0000000504c00000| Complete +| 383|0x0000000504e00000, 0x0000000505000000, 0x0000000505000000|100%| E|CS|TAMS 0x0000000504e00000, 0x0000000504e00000| Complete +| 384|0x0000000505000000, 0x0000000505200000, 0x0000000505200000|100%| E|CS|TAMS 0x0000000505000000, 0x0000000505000000| Complete +| 385|0x0000000505200000, 0x0000000505400000, 0x0000000505400000|100%| E|CS|TAMS 0x0000000505200000, 0x0000000505200000| Complete +| 386|0x0000000505400000, 0x0000000505600000, 0x0000000505600000|100%| E|CS|TAMS 0x0000000505400000, 0x0000000505400000| Complete +| 387|0x0000000505600000, 0x0000000505800000, 0x0000000505800000|100%| E|CS|TAMS 0x0000000505600000, 0x0000000505600000| Complete +| 388|0x0000000505800000, 0x0000000505a00000, 0x0000000505a00000|100%| E|CS|TAMS 0x0000000505800000, 0x0000000505800000| Complete +| 389|0x0000000505a00000, 0x0000000505c00000, 0x0000000505c00000|100%| E|CS|TAMS 0x0000000505a00000, 0x0000000505a00000| Complete +| 390|0x0000000505c00000, 0x0000000505e00000, 0x0000000505e00000|100%| E|CS|TAMS 0x0000000505c00000, 0x0000000505c00000| Complete +| 391|0x0000000505e00000, 0x0000000506000000, 0x0000000506000000|100%| E|CS|TAMS 0x0000000505e00000, 0x0000000505e00000| Complete +| 392|0x0000000506000000, 0x0000000506200000, 0x0000000506200000|100%| E|CS|TAMS 0x0000000506000000, 0x0000000506000000| Complete +| 393|0x0000000506200000, 0x0000000506400000, 0x0000000506400000|100%| E|CS|TAMS 0x0000000506200000, 0x0000000506200000| Complete +| 394|0x0000000506400000, 0x0000000506600000, 0x0000000506600000|100%| E|CS|TAMS 0x0000000506400000, 0x0000000506400000| Complete +| 395|0x0000000506600000, 0x0000000506800000, 0x0000000506800000|100%| E|CS|TAMS 0x0000000506600000, 0x0000000506600000| Complete +| 396|0x0000000506800000, 0x0000000506a00000, 0x0000000506a00000|100%| E|CS|TAMS 0x0000000506800000, 0x0000000506800000| Complete +| 397|0x0000000506a00000, 0x0000000506c00000, 0x0000000506c00000|100%| E|CS|TAMS 0x0000000506a00000, 0x0000000506a00000| Complete +| 398|0x0000000506c00000, 0x0000000506e00000, 0x0000000506e00000|100%| E|CS|TAMS 0x0000000506c00000, 0x0000000506c00000| Complete +| 399|0x0000000506e00000, 0x0000000507000000, 0x0000000507000000|100%| E|CS|TAMS 0x0000000506e00000, 0x0000000506e00000| Complete +| 400|0x0000000507000000, 0x0000000507200000, 0x0000000507200000|100%| E|CS|TAMS 0x0000000507000000, 0x0000000507000000| Complete +| 401|0x0000000507200000, 0x0000000507400000, 0x0000000507400000|100%| E|CS|TAMS 0x0000000507200000, 0x0000000507200000| Complete +| 402|0x0000000507400000, 0x0000000507600000, 0x0000000507600000|100%| E|CS|TAMS 0x0000000507400000, 0x0000000507400000| Complete +| 403|0x0000000507600000, 0x0000000507800000, 0x0000000507800000|100%| E|CS|TAMS 0x0000000507600000, 0x0000000507600000| Complete +| 404|0x0000000507800000, 0x0000000507a00000, 0x0000000507a00000|100%| E|CS|TAMS 0x0000000507800000, 0x0000000507800000| Complete +| 405|0x0000000507a00000, 0x0000000507c00000, 0x0000000507c00000|100%| E|CS|TAMS 0x0000000507a00000, 0x0000000507a00000| Complete +| 406|0x0000000507c00000, 0x0000000507e00000, 0x0000000507e00000|100%| E|CS|TAMS 0x0000000507c00000, 0x0000000507c00000| Complete +| 407|0x0000000507e00000, 0x0000000508000000, 0x0000000508000000|100%| E|CS|TAMS 0x0000000507e00000, 0x0000000507e00000| Complete +| 408|0x0000000508000000, 0x0000000508200000, 0x0000000508200000|100%| E|CS|TAMS 0x0000000508000000, 0x0000000508000000| Complete +| 409|0x0000000508200000, 0x0000000508400000, 0x0000000508400000|100%| E|CS|TAMS 0x0000000508200000, 0x0000000508200000| Complete +| 410|0x0000000508400000, 0x0000000508600000, 0x0000000508600000|100%| E|CS|TAMS 0x0000000508400000, 0x0000000508400000| Complete +| 411|0x0000000508600000, 0x0000000508800000, 0x0000000508800000|100%| E|CS|TAMS 0x0000000508600000, 0x0000000508600000| Complete +| 412|0x0000000508800000, 0x0000000508a00000, 0x0000000508a00000|100%| E|CS|TAMS 0x0000000508800000, 0x0000000508800000| Complete +| 413|0x0000000508a00000, 0x0000000508c00000, 0x0000000508c00000|100%| E|CS|TAMS 0x0000000508a00000, 0x0000000508a00000| Complete +| 414|0x0000000508c00000, 0x0000000508e00000, 0x0000000508e00000|100%| E|CS|TAMS 0x0000000508c00000, 0x0000000508c00000| Complete +| 415|0x0000000508e00000, 0x0000000509000000, 0x0000000509000000|100%| E|CS|TAMS 0x0000000508e00000, 0x0000000508e00000| Complete +| 416|0x0000000509000000, 0x0000000509200000, 0x0000000509200000|100%| E|CS|TAMS 0x0000000509000000, 0x0000000509000000| Complete +| 417|0x0000000509200000, 0x0000000509400000, 0x0000000509400000|100%| E|CS|TAMS 0x0000000509200000, 0x0000000509200000| Complete +| 418|0x0000000509400000, 0x0000000509600000, 0x0000000509600000|100%| E|CS|TAMS 0x0000000509400000, 0x0000000509400000| Complete +| 419|0x0000000509600000, 0x0000000509800000, 0x0000000509800000|100%| E|CS|TAMS 0x0000000509600000, 0x0000000509600000| Complete +| 420|0x0000000509800000, 0x0000000509a00000, 0x0000000509a00000|100%| E|CS|TAMS 0x0000000509800000, 0x0000000509800000| Complete +| 421|0x0000000509a00000, 0x0000000509c00000, 0x0000000509c00000|100%| E|CS|TAMS 0x0000000509a00000, 0x0000000509a00000| Complete +| 422|0x0000000509c00000, 0x0000000509e00000, 0x0000000509e00000|100%| E|CS|TAMS 0x0000000509c00000, 0x0000000509c00000| Complete +| 423|0x0000000509e00000, 0x000000050a000000, 0x000000050a000000|100%| E|CS|TAMS 0x0000000509e00000, 0x0000000509e00000| Complete +| 424|0x000000050a000000, 0x000000050a200000, 0x000000050a200000|100%| E|CS|TAMS 0x000000050a000000, 0x000000050a000000| Complete +| 425|0x000000050a200000, 0x000000050a400000, 0x000000050a400000|100%| E|CS|TAMS 0x000000050a200000, 0x000000050a200000| Complete +| 426|0x000000050a400000, 0x000000050a600000, 0x000000050a600000|100%| E|CS|TAMS 0x000000050a400000, 0x000000050a400000| Complete +| 427|0x000000050a600000, 0x000000050a800000, 0x000000050a800000|100%| E|CS|TAMS 0x000000050a600000, 0x000000050a600000| Complete +| 428|0x000000050a800000, 0x000000050aa00000, 0x000000050aa00000|100%| E|CS|TAMS 0x000000050a800000, 0x000000050a800000| Complete +| 429|0x000000050aa00000, 0x000000050ac00000, 0x000000050ac00000|100%| E|CS|TAMS 0x000000050aa00000, 0x000000050aa00000| Complete +| 430|0x000000050ac00000, 0x000000050ae00000, 0x000000050ae00000|100%| E|CS|TAMS 0x000000050ac00000, 0x000000050ac00000| Complete +| 431|0x000000050ae00000, 0x000000050b000000, 0x000000050b000000|100%| E|CS|TAMS 0x000000050ae00000, 0x000000050ae00000| Complete +| 432|0x000000050b000000, 0x000000050b200000, 0x000000050b200000|100%| E|CS|TAMS 0x000000050b000000, 0x000000050b000000| Complete +| 433|0x000000050b200000, 0x000000050b400000, 0x000000050b400000|100%| E|CS|TAMS 0x000000050b200000, 0x000000050b200000| Complete +| 434|0x000000050b400000, 0x000000050b600000, 0x000000050b600000|100%| E|CS|TAMS 0x000000050b400000, 0x000000050b400000| Complete +| 435|0x000000050b600000, 0x000000050b800000, 0x000000050b800000|100%| E|CS|TAMS 0x000000050b600000, 0x000000050b600000| Complete +| 436|0x000000050b800000, 0x000000050ba00000, 0x000000050ba00000|100%| E|CS|TAMS 0x000000050b800000, 0x000000050b800000| Complete +| 437|0x000000050ba00000, 0x000000050bc00000, 0x000000050bc00000|100%| E|CS|TAMS 0x000000050ba00000, 0x000000050ba00000| Complete +| 438|0x000000050bc00000, 0x000000050be00000, 0x000000050be00000|100%| E|CS|TAMS 0x000000050bc00000, 0x000000050bc00000| Complete +| 439|0x000000050be00000, 0x000000050c000000, 0x000000050c000000|100%| E|CS|TAMS 0x000000050be00000, 0x000000050be00000| Complete +| 440|0x000000050c000000, 0x000000050c200000, 0x000000050c200000|100%| E|CS|TAMS 0x000000050c000000, 0x000000050c000000| Complete +| 441|0x000000050c200000, 0x000000050c400000, 0x000000050c400000|100%| E|CS|TAMS 0x000000050c200000, 0x000000050c200000| Complete +| 442|0x000000050c400000, 0x000000050c600000, 0x000000050c600000|100%| E|CS|TAMS 0x000000050c400000, 0x000000050c400000| Complete +| 443|0x000000050c600000, 0x000000050c800000, 0x000000050c800000|100%| E|CS|TAMS 0x000000050c600000, 0x000000050c600000| Complete +| 444|0x000000050c800000, 0x000000050ca00000, 0x000000050ca00000|100%| E|CS|TAMS 0x000000050c800000, 0x000000050c800000| Complete +| 445|0x000000050ca00000, 0x000000050cc00000, 0x000000050cc00000|100%| E|CS|TAMS 0x000000050ca00000, 0x000000050ca00000| Complete +| 446|0x000000050cc00000, 0x000000050ce00000, 0x000000050ce00000|100%| E|CS|TAMS 0x000000050cc00000, 0x000000050cc00000| Complete +| 447|0x000000050ce00000, 0x000000050d000000, 0x000000050d000000|100%| E|CS|TAMS 0x000000050ce00000, 0x000000050ce00000| Complete +| 448|0x000000050d000000, 0x000000050d200000, 0x000000050d200000|100%| E|CS|TAMS 0x000000050d000000, 0x000000050d000000| Complete +| 449|0x000000050d200000, 0x000000050d400000, 0x000000050d400000|100%| E|CS|TAMS 0x000000050d200000, 0x000000050d200000| Complete +| 450|0x000000050d400000, 0x000000050d600000, 0x000000050d600000|100%| E|CS|TAMS 0x000000050d400000, 0x000000050d400000| Complete +| 451|0x000000050d600000, 0x000000050d800000, 0x000000050d800000|100%| E|CS|TAMS 0x000000050d600000, 0x000000050d600000| Complete +| 452|0x000000050d800000, 0x000000050da00000, 0x000000050da00000|100%| E|CS|TAMS 0x000000050d800000, 0x000000050d800000| Complete +| 453|0x000000050da00000, 0x000000050dc00000, 0x000000050dc00000|100%| E|CS|TAMS 0x000000050da00000, 0x000000050da00000| Complete +| 454|0x000000050dc00000, 0x000000050de00000, 0x000000050de00000|100%| E|CS|TAMS 0x000000050dc00000, 0x000000050dc00000| Complete +| 455|0x000000050de00000, 0x000000050e000000, 0x000000050e000000|100%| E|CS|TAMS 0x000000050de00000, 0x000000050de00000| Complete +| 456|0x000000050e000000, 0x000000050e200000, 0x000000050e200000|100%| E|CS|TAMS 0x000000050e000000, 0x000000050e000000| Complete +| 457|0x000000050e200000, 0x000000050e400000, 0x000000050e400000|100%| E|CS|TAMS 0x000000050e200000, 0x000000050e200000| Complete +| 458|0x000000050e400000, 0x000000050e600000, 0x000000050e600000|100%| E|CS|TAMS 0x000000050e400000, 0x000000050e400000| Complete +| 459|0x000000050e600000, 0x000000050e800000, 0x000000050e800000|100%| E|CS|TAMS 0x000000050e600000, 0x000000050e600000| Complete +| 460|0x000000050e800000, 0x000000050ea00000, 0x000000050ea00000|100%| E|CS|TAMS 0x000000050e800000, 0x000000050e800000| Complete +| 461|0x000000050ea00000, 0x000000050ec00000, 0x000000050ec00000|100%| E|CS|TAMS 0x000000050ea00000, 0x000000050ea00000| Complete +| 462|0x000000050ec00000, 0x000000050ee00000, 0x000000050ee00000|100%| E|CS|TAMS 0x000000050ec00000, 0x000000050ec00000| Complete +| 463|0x000000050ee00000, 0x000000050f000000, 0x000000050f000000|100%| E|CS|TAMS 0x000000050ee00000, 0x000000050ee00000| Complete +| 464|0x000000050f000000, 0x000000050f200000, 0x000000050f200000|100%| E|CS|TAMS 0x000000050f000000, 0x000000050f000000| Complete +| 767|0x0000000534e00000, 0x0000000535000000, 0x0000000535000000|100%| E|CS|TAMS 0x0000000534e00000, 0x0000000534e00000| Complete +|2046|0x00000005d4c00000, 0x00000005d4d74000, 0x00000005d4e00000| 72%|OA| |TAMS 0x00000005d4d74000, 0x00000005d4c00000| Untracked +|2047|0x00000005d4e00000, 0x00000005d4e74000, 0x00000005d5000000| 22%|CA| |TAMS 0x00000005d4e74000, 0x00000005d4e00000| Untracked + +Card table byte_map: [0x000000011089c000,0x000000011109c000] _byte_map_base: 0x000000010e1f4000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x0000000afccc4510, (CMBitMap*) 0x0000000afccc4550 + Prev Bits: [0x000000011189c000, 0x000000011589c000) + Next Bits: [0x000000011589c000, 0x000000011989c000) + +Polling page: 0x00000001021a4000 + +Metaspace: + +Usage: + Non-class: 129.98 MB used. + Class: 17.71 MB used. + Both: 147.69 MB used. + +Virtual space: + Non-class space: 192.00 MB reserved, 130.56 MB ( 68%) committed, 3 nodes. + Class space: 1.00 GB reserved, 18.19 MB ( 2%) committed, 1 nodes. + Both: 1.19 GB reserved, 148.75 MB ( 12%) committed. + +Chunk freelists: + Non-Class: 12.84 MB + Class: 13.85 MB + Both: 26.69 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 217.12 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 12. +num_arena_births: 2134. +num_arena_deaths: 412. +num_vsnodes_births: 4. +num_vsnodes_deaths: 0. +num_space_committed: 2377. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 582. +num_chunks_taken_from_freelist: 8069. +num_chunk_merges: 221. +num_chunk_splits: 5784. +num_chunks_enlarged: 4367. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=119168Kb used=10598Kb max_used=10598Kb free=108569Kb + bounds [0x00000001249a4000, 0x0000000125404000, 0x000000012be04000] +CodeHeap 'profiled nmethods': size=119152Kb used=21762Kb max_used=21762Kb free=97389Kb + bounds [0x000000011ce04000, 0x000000011e354000, 0x0000000124260000] +CodeHeap 'non-nmethods': size=7440Kb used=3320Kb max_used=3502Kb free=4119Kb + bounds [0x0000000124260000, 0x00000001245e0000, 0x00000001249a4000] + total_blobs=13178 nmethods=12116 adapters=973 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 21.310 Thread 0x0000000afb471200 16525 3 scala.collection.mutable.IndexedSeq::companion (13 bytes) +Event: 21.310 Thread 0x0000000af816aa00 16526 3 java.util.Collections::synchronizedSet (10 bytes) +Event: 21.310 Thread 0x0000000afb471200 nmethod 16525 0x000000011e343a10 code [0x000000011e343bc0, 0x000000011e343d48] +Event: 21.310 Thread 0x0000000afb471200 16527 3 org.apache.spark.util.Utils$::bytesToString (146 bytes) +Event: 21.310 Thread 0x0000000afcd82400 nmethod 16523 0x000000011e343e90 code [0x000000011e344080, 0x000000011e344558] +Event: 21.310 Thread 0x0000000afcd82400 16529 2 scala.math.Numeric::zero$ (13 bytes) +Event: 21.310 Thread 0x0000000afb470c00 nmethod 16524 0x000000011e344790 code [0x000000011e3449c0, 0x000000011e344fb8] +Event: 21.310 Thread 0x0000000afcd82400 nmethod 16529 0x000000011e345210 code [0x000000011e3453c0, 0x000000011e345578] +Event: 21.310 Thread 0x0000000afcd82400 16528 3 java.util.regex.Pattern::createGroup (86 bytes) +Event: 21.310 Thread 0x0000000afb470c00 16530 3 org.apache.logging.log4j.core.impl.JdkMapAdapterStringMap:: (35 bytes) +Event: 21.310 Thread 0x0000000af816aa00 nmethod 16526 0x000000011e345710 code [0x000000011e345940, 0x000000011e345fb8] +Event: 21.310 Thread 0x0000000af816aa00 16532 3 java.util.regex.Pattern::accept (43 bytes) +Event: 21.311 Thread 0x0000000afb470c00 nmethod 16530 0x000000011e346210 code [0x000000011e346400, 0x000000011e3468f8] +Event: 21.311 Thread 0x0000000afb470c00 16531 3 org.apache.logging.log4j.core.impl.JdkMapAdapterStringMap::freeze (15 bytes) +Event: 21.311 Thread 0x0000000af816aa00 nmethod 16532 0x000000011e346b10 code [0x000000011e346d00, 0x000000011e347038] +Event: 21.311 Thread 0x0000000af816aa00 16533 3 org.apache.logging.log4j.core.impl.ThreadContextDataInjector$ForDefaultThreadContextMap::frozenStringMap (29 bytes) +Event: 21.311 Thread 0x0000000afcd82400 nmethod 16528 0x000000011e347190 code [0x000000011e3473c0, 0x000000011e347b58] +Event: 21.311 Thread 0x0000000afb471200 nmethod 16527 0x000000011e347e10 code [0x000000011e3480c0, 0x000000011e348b38] +Event: 21.311 Thread 0x0000000afb470c00 nmethod 16531 0x000000011e348f10 code [0x000000011e3490c0, 0x000000011e349248] +Event: 21.311 Thread 0x0000000af816aa00 nmethod 16533 0x000000011e349390 code [0x000000011e349580, 0x000000011e3497e8] + +GC Heap History (20 events): +Event: 6.465 GC heap before +{Heap before GC invocations=27 (full 0): + garbage-first heap total 342016K, used 303817K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 96 young (196608K), 1 survivors (2048K) + Metaspace used 118486K, committed 119232K, reserved 1179648K + class space used 14224K, committed 14592K, reserved 1048576K +} +Event: 6.468 GC heap after +{Heap after GC invocations=28 (full 0): + garbage-first heap total 342016K, used 125195K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 11 young (22528K), 11 survivors (22528K) + Metaspace used 118486K, committed 119232K, reserved 1179648K + class space used 14224K, committed 14592K, reserved 1048576K +} +Event: 6.602 GC heap before +{Heap before GC invocations=28 (full 0): + garbage-first heap total 342016K, used 209163K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 54 young (110592K), 11 survivors (22528K) + Metaspace used 120736K, committed 121600K, reserved 1179648K + class space used 14577K, committed 14976K, reserved 1048576K +} +Event: 6.606 GC heap after +{Heap after GC invocations=29 (full 0): + garbage-first heap total 342016K, used 133061K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 120736K, committed 121600K, reserved 1179648K + class space used 14577K, committed 14976K, reserved 1048576K +} +Event: 6.855 GC heap before +{Heap before GC invocations=30 (full 0): + garbage-first heap total 342016K, used 276421K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 70 young (143360K), 4 survivors (8192K) + Metaspace used 124828K, committed 125696K, reserved 1179648K + class space used 14919K, committed 15360K, reserved 1048576K +} +Event: 6.856 GC heap after +{Heap after GC invocations=31 (full 0): + garbage-first heap total 342016K, used 147033K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 6 young (12288K), 6 survivors (12288K) + Metaspace used 124828K, committed 125696K, reserved 1179648K + class space used 14919K, committed 15360K, reserved 1048576K +} +Event: 7.137 GC heap before +{Heap before GC invocations=31 (full 0): + garbage-first heap total 342016K, used 274009K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 69 young (141312K), 6 survivors (12288K) + Metaspace used 129032K, committed 129920K, reserved 1179648K + class space used 15269K, committed 15680K, reserved 1048576K +} +Event: 7.139 GC heap after +{Heap after GC invocations=32 (full 0): + garbage-first heap total 342016K, used 148364K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 4 young (8192K), 4 survivors (8192K) + Metaspace used 129032K, committed 129920K, reserved 1179648K + class space used 15269K, committed 15680K, reserved 1048576K +} +Event: 7.199 GC heap before +{Heap before GC invocations=33 (full 0): + garbage-first heap total 342016K, used 195468K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 28 young (57344K), 4 survivors (8192K) + Metaspace used 130550K, committed 131328K, reserved 1179648K + class space used 15456K, committed 15808K, reserved 1048576K +} +Event: 7.201 GC heap after +{Heap after GC invocations=34 (full 0): + garbage-first heap total 342016K, used 148042K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 130550K, committed 131328K, reserved 1179648K + class space used 15456K, committed 15808K, reserved 1048576K +} +Event: 7.326 GC heap before +{Heap before GC invocations=35 (full 0): + garbage-first heap total 342016K, used 201290K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 26 young (53248K), 3 survivors (6144K) + Metaspace used 132612K, committed 133376K, reserved 1179648K + class space used 15837K, committed 16192K, reserved 1048576K +} +Event: 7.328 GC heap after +{Heap after GC invocations=36 (full 0): + garbage-first heap total 342016K, used 144714K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 132612K, committed 133376K, reserved 1179648K + class space used 15837K, committed 16192K, reserved 1048576K +} +Event: 7.565 GC heap before +{Heap before GC invocations=37 (full 0): + garbage-first heap total 342016K, used 273738K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 69 young (141312K), 2 survivors (4096K) + Metaspace used 139445K, committed 140224K, reserved 1179648K + class space used 16694K, committed 17024K, reserved 1048576K +} +Event: 7.567 GC heap after +{Heap after GC invocations=38 (full 0): + garbage-first heap total 342016K, used 125903K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 139445K, committed 140224K, reserved 1179648K + class space used 16694K, committed 17024K, reserved 1048576K +} +Event: 7.580 GC heap before +{Heap before GC invocations=38 (full 0): + garbage-first heap total 342016K, used 136143K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 8 young (16384K), 3 survivors (6144K) + Metaspace used 140017K, committed 140736K, reserved 1179648K + class space used 16762K, committed 17088K, reserved 1048576K +} +Event: 7.582 GC heap after +{Heap after GC invocations=39 (full 0): + garbage-first heap total 342016K, used 124142K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 1 young (2048K), 1 survivors (2048K) + Metaspace used 140017K, committed 140736K, reserved 1179648K + class space used 16762K, committed 17088K, reserved 1048576K +} +Event: 7.851 GC heap before +{Heap before GC invocations=39 (full 0): + garbage-first heap total 342016K, used 287982K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 78 young (159744K), 1 survivors (2048K) + Metaspace used 148180K, committed 149120K, reserved 1245184K + class space used 17685K, committed 18112K, reserved 1048576K +} +Event: 7.853 GC heap after +{Heap after GC invocations=40 (full 0): + garbage-first heap total 958464K, used 126026K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 148180K, committed 149120K, reserved 1245184K + class space used 17685K, committed 18112K, reserved 1048576K +} +Event: 9.193 GC heap before +{Heap before GC invocations=40 (full 0): + garbage-first heap total 958464K, used 695370K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 280 young (573440K), 2 survivors (4096K) + Metaspace used 148181K, committed 149120K, reserved 1245184K + class space used 17686K, committed 18112K, reserved 1048576K +} +Event: 9.194 GC heap after +{Heap after GC invocations=41 (full 0): + garbage-first heap total 958464K, used 126097K [0x00000004d5000000, 0x00000005d5000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 148181K, committed 149120K, reserved 1245184K + class space used 17686K, committed 18112K, reserved 1048576K +} + +Dll operation events (20 events): +Event: 3.646 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li +Event: 3.646 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), +Event: 3.646 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil +Event: 5.628 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib +Event: 6.130 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach +Event: 6.130 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa +Event: 6.130 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre +Event: 6.130 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ +Event: 6.130 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex +Event: 6.130 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java +Event: 6.130 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 6.130 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C +Event: 6.130 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex +Event: 6.130 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp +Event: 6.130 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), +Event: 6.131 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil +Event: 6.131 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su +Event: 6.131 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no +Event: 6.278 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15423878368099207039.dylib +Event: 7.020 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-afedc8a6-4493-4c3c-aed3-c280414ad816-libsnappyjava.dylib + +Deoptimization events (20 events): +Event: 21.297 Thread 0x0000000afb041800 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c +Event: 21.297 Thread 0x0000000afb041800 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 +Event: 21.297 Thread 0x0000000afb041800 DEOPT PACKING pc=0x000000012537452c sp=0x000000017ec661c0 +Event: 21.297 Thread 0x0000000afb041800 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017ec66160 mode 2 +Event: 21.297 Thread 0x0000000afb042400 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c +Event: 21.297 Thread 0x0000000afb042400 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 +Event: 21.297 Thread 0x0000000afb042400 DEOPT PACKING pc=0x000000012537452c sp=0x000000017f47e1c0 +Event: 21.297 Thread 0x0000000afb042400 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f47e160 mode 2 +Event: 21.297 Thread 0x0000000af816a400 Uncommon trap: trap_request=0xffffff66 fr.pc=0x000000012538bb20 relative=0x00000000000024e0 +Event: 21.297 Thread 0x0000000af816a400 Uncommon trap: reason=speculate_class_check action=maybe_recompile pc=0x000000012538bb20 method=java.io.ObjectInputStream.readOrdinaryObject(Z)Ljava/lang/Object; @ 226 c2 +Event: 21.297 Thread 0x0000000af816a400 DEOPT PACKING pc=0x000000012538bb20 sp=0x000000017fea2260 +Event: 21.297 Thread 0x0000000af816a400 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017fea2220 mode 2 +Event: 21.298 Thread 0x0000000afb041e00 Uncommon trap: trap_request=0xffffff66 fr.pc=0x000000012538bb20 relative=0x00000000000024e0 +Event: 21.298 Thread 0x0000000afb041e00 Uncommon trap: reason=speculate_class_check action=maybe_recompile pc=0x000000012538bb20 method=java.io.ObjectInputStream.readOrdinaryObject(Z)Ljava/lang/Object; @ 226 c2 +Event: 21.298 Thread 0x0000000afb041e00 DEOPT PACKING pc=0x000000012538bb20 sp=0x000000017f06ffd0 +Event: 21.298 Thread 0x0000000afb041e00 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f06ff90 mode 2 +Event: 21.298 Thread 0x0000000afb042a00 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c +Event: 21.298 Thread 0x0000000afb042a00 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 +Event: 21.298 Thread 0x0000000afb042a00 DEOPT PACKING pc=0x000000012537452c sp=0x000000017f88a1c0 +Event: 21.298 Thread 0x0000000afb042a00 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f88a160 mode 2 + +Classes loaded (20 events): +Event: 7.732 Loading class java/lang/Class$EnclosingMethodInfo +Event: 7.732 Loading class java/lang/Class$EnclosingMethodInfo done +Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper +Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$Unmapper +Event: 16.521 Loading class jdk/internal/access/foreign/UnmapperProxy +Event: 16.521 Loading class jdk/internal/access/foreign/UnmapperProxy done +Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$Unmapper done +Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done +Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask +Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask +Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done +Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done +Event: 21.095 Loading class java/util/function/LongBinaryOperator +Event: 21.095 Loading class java/util/function/LongBinaryOperator done +Event: 21.107 Loading class sun/nio/ch/ChannelInputStream +Event: 21.107 Loading class sun/nio/ch/ChannelInputStream done +Event: 21.179 Loading class java/math/BigDecimal$StringBuilderHelper +Event: 21.179 Loading class java/math/BigDecimal$StringBuilderHelper done +Event: 21.241 Loading class java/nio/channels/Channels$ReadableByteChannelImpl +Event: 21.241 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done + +Classes unloaded (20 events): +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3c000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3c000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b800 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b800' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b400 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b400' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3ac00 'java/lang/invoke/LambdaForm$MH+0x000000d001e3ac00' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3a800 'java/lang/invoke/LambdaForm$MH+0x000000d001e3a800' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3a000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3a000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e39c00' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39800 'java/lang/invoke/LambdaForm$MH+0x000000d001e39800' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39000 'java/lang/invoke/LambdaForm$MH+0x000000d001e39000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e38c00' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38800 'java/lang/invoke/LambdaForm$MH+0x000000d001e38800' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38000 'java/lang/invoke/LambdaForm$MH+0x000000d001e38000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e37c00' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37400 'java/lang/invoke/LambdaForm$MH+0x000000d001e37400' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37800 'java/lang/invoke/LambdaForm$MH+0x000000d001e37800' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37000 'java/lang/invoke/LambdaForm$MH+0x000000d001e37000' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36400 'java/lang/invoke/LambdaForm$MH+0x000000d001e36400' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e36c00' +Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36800 'java/lang/invoke/LambdaForm$MH+0x000000d001e36800' + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 21.237 Thread 0x0000000afb042400 Exception (0x0000000502c9d4e0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.237 Thread 0x0000000afb041e00 Exception (0x0000000502a3b288) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.237 Thread 0x0000000afb042a00 Exception (0x0000000502b2d308) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.237 Thread 0x0000000afb041800 Exception (0x0000000502de8160) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.237 Thread 0x0000000afb041200 Exception (0x0000000503390408) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b41728) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033dd288) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502dfc750) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb20d8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041e00 Exception (0x0000000502846808) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b455b8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502900ea0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033e1238) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb5ec8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041e00 Exception (0x000000050284a5e0) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b48e98) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502904730) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033e4ac8) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb041e00 Exception (0x000000050284de70) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] +Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb9758) +thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] + +VM Operations (20 events): +Event: 7.989 Executing VM operation: HandshakeAllThreads +Event: 7.989 Executing VM operation: HandshakeAllThreads done +Event: 7.995 Executing VM operation: ICBufferFull +Event: 7.995 Executing VM operation: ICBufferFull done +Event: 8.996 Executing VM operation: Cleanup +Event: 8.996 Executing VM operation: Cleanup done +Event: 9.193 Executing VM operation: G1CollectForAllocation +Event: 9.194 Executing VM operation: G1CollectForAllocation done +Event: 12.209 Executing VM operation: Cleanup +Event: 12.209 Executing VM operation: Cleanup done +Event: 17.234 Executing VM operation: Cleanup +Event: 17.234 Executing VM operation: Cleanup done +Event: 21.127 Executing VM operation: ICBufferFull +Event: 21.127 Executing VM operation: ICBufferFull done +Event: 21.185 Executing VM operation: ICBufferFull +Event: 21.185 Executing VM operation: ICBufferFull done +Event: 21.240 Executing VM operation: ICBufferFull +Event: 21.240 Executing VM operation: ICBufferFull done +Event: 21.277 Executing VM operation: ICBufferFull +Event: 21.277 Executing VM operation: ICBufferFull done + +Memory protections (20 events): +Event: 9.544 Protecting memory [0x0000000304b30000,0x0000000304b3c000] with protection modes 3 +Event: 9.544 Protecting memory [0x0000000300c34000,0x0000000300c40000] with protection modes 3 +Event: 9.545 Protecting memory [0x0000000300a28000,0x0000000300a34000] with protection modes 3 +Event: 9.545 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 3 +Event: 9.557 Protecting memory [0x000000017e240000,0x000000017e24c000] with protection modes 3 +Event: 14.562 Protecting memory [0x0000000174c38000,0x0000000174c44000] with protection modes 3 +Event: 14.562 Protecting memory [0x00000001741fc000,0x0000000174208000] with protection modes 3 +Event: 14.562 Protecting memory [0x0000000174e44000,0x0000000174e50000] with protection modes 3 +Event: 16.522 Protecting memory [0x0000000173ff0000,0x0000000173ffc000] with protection modes 3 +Event: 21.113 Protecting memory [0x0000000173ff0000,0x0000000173ffc000] with protection modes 0 +Event: 21.113 Protecting memory [0x00000001741fc000,0x0000000174208000] with protection modes 0 +Event: 21.114 Protecting memory [0x0000000174c38000,0x0000000174c44000] with protection modes 0 +Event: 21.114 Protecting memory [0x0000000174e44000,0x0000000174e50000] with protection modes 0 +Event: 21.114 Protecting memory [0x000000017e240000,0x000000017e24c000] with protection modes 0 +Event: 21.169 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 0 +Event: 21.222 Protecting memory [0x000000017f894000,0x000000017f8a0000] with protection modes 0 +Event: 21.254 Protecting memory [0x0000000300c28000,0x0000000300c34000] with protection modes 0 +Event: 21.260 Protecting memory [0x00000003040f4000,0x0000000304100000] with protection modes 0 +Event: 21.261 Protecting memory [0x000000030450c000,0x0000000304518000] with protection modes 0 +Event: 21.265 Protecting memory [0x0000000304718000,0x0000000304724000] with protection modes 0 + +Nmethod flushes (20 events): +Event: 7.362 Thread 0x0000000afcd82a00 flushing osr nmethod 0x000000011dccf790 +Event: 7.362 Thread 0x0000000afcd82a00 flushing osr nmethod 0x000000011dcd1190 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dcd2b90 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dd3e910 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dd4cb90 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd7a10 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd8190 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd8a90 +Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd9490 +Event: 7.991 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ce31990 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d095f90 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d1be990 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2c1310 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2c2910 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2cd910 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d31ab90 +Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d32d490 +Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d4b0790 +Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d5c2f90 +Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d60e790 + +Events (20 events): +Event: 9.544 Thread 0x0000000af68a5800 Thread exited: 0x0000000af68a5800 +Event: 9.544 Thread 0x0000000af6861800 Thread exited: 0x0000000af6861800 +Event: 9.545 Thread 0x0000000af6861200 Thread exited: 0x0000000af6861200 +Event: 9.545 Thread 0x0000000af6860c00 Thread exited: 0x0000000af6860c00 +Event: 9.557 Thread 0x0000000af6860000 Thread exited: 0x0000000af6860000 +Event: 14.562 Thread 0x0000000afb471200 Thread exited: 0x0000000afb471200 +Event: 14.563 Thread 0x0000000afb470c00 Thread exited: 0x0000000afb470c00 +Event: 14.563 Thread 0x0000000af816aa00 Thread exited: 0x0000000af816aa00 +Event: 16.522 Thread 0x0000000af8169800 Thread exited: 0x0000000af8169800 +Event: 21.113 Thread 0x0000000afcd82400 Thread added: 0x0000000af68a7600 +Event: 21.113 Thread 0x0000000afcd82400 Thread added: 0x0000000af8169800 +Event: 21.114 Thread 0x0000000afcd82400 Thread added: 0x0000000af816aa00 +Event: 21.114 Thread 0x0000000af816aa00 Thread added: 0x0000000afb470c00 +Event: 21.114 Thread 0x0000000af816aa00 Thread added: 0x0000000afb471200 +Event: 21.169 Thread 0x0000000afcd83600 Thread added: 0x0000000af6860c00 +Event: 21.222 Thread 0x0000000af816aa00 Thread added: 0x0000000af5631e00 +Event: 21.254 Thread 0x0000000af816aa00 Thread added: 0x0000000af601c600 +Event: 21.260 Thread 0x0000000afb471200 Thread added: 0x0000000af601cc00 +Event: 21.261 Thread 0x0000000afb471200 Thread added: 0x0000000af601d200 +Event: 21.265 Thread 0x0000000afcd82400 Thread added: 0x0000000af601d800 + + +Dynamic libraries: +0x00000001021bc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib +0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa +0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit +0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData +0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation +0x000000019c2cf000 /usr/lib/libSystem.B.dylib +0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation +0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore +0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap +0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport +0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity +0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard +0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard +0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices +0x000000028a849000 /usr/lib/libRosetta.dylib +0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport +0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore +0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools +0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement +0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration +0x000000019df6a000 /usr/lib/libspindump.dylib +0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers +0x000000019b566000 /usr/lib/libbsm.0.dylib +0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib +0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics +0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout +0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal +0x0000000199fb3000 /usr/lib/liblangid.dylib +0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG +0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight +0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine +0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics +0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary +0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate +0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices +0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface +0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib +0x000000019c218000 /usr/lib/libz.1.dylib +0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices +0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation +0x00000001904e7000 /usr/lib/libicucore.A.dylib +0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox +0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore +0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle +0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput +0x0000000192f23000 /usr/lib/libMobileGestalt.dylib +0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox +0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore +0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security +0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition +0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI +0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio +0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration +0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport +0x0000000192f21000 /usr/lib/libenergytrace.dylib +0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox +0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit +0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices +0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis +0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL +0x0000000196db8000 /usr/lib/libxml2.2.dylib +0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag +0x000000018c3a8000 /usr/lib/libobjc.A.dylib +0x000000018c70e000 /usr/lib/libc++.1.dylib +0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility +0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync +0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation +0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage +0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText +0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable +0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection +0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport +0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO +0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols +0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph +0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices +0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags +0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures +0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking +0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib +0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib +0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib +0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib +0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib +0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib +0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib +0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib +0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib +0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib +0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib +0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib +0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib +0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib +0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib +0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib +0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib +0x00000001a9390000 /usr/lib/swift/libswiftos.dylib +0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib +0x000000019c4fb000 /usr/lib/libcompression.dylib +0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO +0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices +0x000000019d9e8000 /usr/lib/libate.dylib +0x000000019c2c9000 /usr/lib/system/libcache.dylib +0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib +0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib +0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib +0x000000018c548000 /usr/lib/system/libcorecrypto.dylib +0x000000018c641000 /usr/lib/system/libdispatch.dylib +0x000000018c3fc000 /usr/lib/system/libdyld.dylib +0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib +0x000000019c267000 /usr/lib/system/libmacho.dylib +0x000000019b53e000 /usr/lib/system/libquarantine.dylib +0x000000019c2bc000 /usr/lib/system/libremovefile.dylib +0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib +0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib +0x000000018c68b000 /usr/lib/system/libsystem_c.dylib +0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib +0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib +0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib +0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib +0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib +0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib +0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib +0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib +0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib +0x000000018c80f000 /usr/lib/system/libsystem_info.dylib +0x000000019c228000 /usr/lib/system/libsystem_m.dylib +0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib +0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib +0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib +0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib +0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib +0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib +0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib +0x000000018c806000 /usr/lib/system/libsystem_platform.dylib +0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib +0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib +0x000000018c527000 /usr/lib/system/libsystem_trace.dylib +0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib +0x000000019c292000 /usr/lib/system/libunwind.dylib +0x000000018c4d1000 /usr/lib/system/libxpc.dylib +0x000000018c7a1000 /usr/lib/libc++abi.dylib +0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib +0x000000019c2d1000 /usr/lib/libfakelink.dylib +0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork +0x000000019c325000 /usr/lib/libarchive.2.dylib +0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine +0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal +0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal +0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal +0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib +0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib +0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib +0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib +0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal +0x000000019c29c000 /usr/lib/liboah.dylib +0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages +0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS +0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents +0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore +0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata +0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices +0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit +0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE +0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices +0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices +0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList +0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib +0x0000000194713000 /usr/lib/libsqlite3.dylib +0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport +0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS +0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices +0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip +0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network +0x000000019c26b000 /usr/lib/system/libkxld.dylib +0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore +0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib +0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity +0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer +0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter +0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport +0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression +0x000000019b54e000 /usr/lib/libcoretls.dylib +0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib +0x000000019c4f4000 /usr/lib/libpam.2.dylib +0x000000019d927000 /usr/lib/libxar.1.dylib +0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS +0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal +0x000000019d936000 /usr/lib/libutil.dylib +0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo +0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer +0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport +0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset +0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog +0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData +0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement +0x0000000192a75000 /usr/lib/libboringssl.dylib +0x0000000194b6e000 /usr/lib/libdns_services.dylib +0x00000001b8104000 /usr/lib/libquic.dylib +0x000000019fbc7000 /usr/lib/libusrtcp.dylib +0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal +0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf +0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib +0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary +0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary +0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams +0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation +0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub +0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport +0x000000019d896000 /usr/lib/liblzma.5.dylib +0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch +0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport +0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect +0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery +0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor +0x00000001bb229000 /usr/lib/libbootpolicy.dylib +0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC +0x00000001c7d55000 /usr/lib/libFDR.dylib +0x00000001cdb6b000 /usr/lib/libamsupport.dylib +0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib +0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport +0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib +0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce +0x000000028a3cd000 /usr/lib/libAppleArchive.dylib +0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib +0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage +0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib +0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib +0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib +0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo +0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS +0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore +0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD +0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy +0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis +0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib +0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices +0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation +0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay +0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox +0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders +0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary +0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator +0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator +0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia +0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC +0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient +0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib +0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib +0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib +0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage +0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary +0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer +0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync +0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL +0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs +0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite +0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime +0x000000019c30a000 /usr/lib/libiconv.2.dylib +0x000000019c266000 /usr/lib/libcharset.1.dylib +0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib +0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets +0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers +0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG +0x000000019dcda000 /usr/lib/libexpat.1.dylib +0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib +0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib +0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib +0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib +0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib +0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib +0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib +0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing +0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib +0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib +0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices +0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing +0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication +0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing +0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager +0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer +0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib +0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib +0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib +0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib +0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib +0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib +0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices +0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG +0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib +0x000000028ad51000 /usr/lib/libhvf.dylib +0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal +0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib +0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore +0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage +0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork +0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix +0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector +0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray +0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions +0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop +0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost +0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools +0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo +0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf +0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter +0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication +0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging +0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols +0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics +0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery +0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation +0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport +0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements +0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit +0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices +0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation +0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering +0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols +0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore +0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession +0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI +0x00000001a0529000 /usr/lib/libAudioStatistics.dylib +0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk +0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib +0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib +0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata +0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy +0x00000001a07e0000 /usr/lib/libSMC.dylib +0x00000001a0949000 /usr/lib/libperfcheck.dylib +0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics +0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib +0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib +0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog +0x00000001b918c000 /usr/lib/libmis.dylib +0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience +0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib +0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore +0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth +0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils +0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID +0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras +0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 +0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth +0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal +0x000000019da79000 /usr/lib/libIOReport.dylib +0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation +0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon +0x000000019b5d9000 /usr/lib/libgermantok.dylib +0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData +0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit +0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording +0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib +0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit +0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory +0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory +0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio +0x00000001b2112000 /usr/lib/libAccessibility.dylib +0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient +0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam +0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration +0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser +0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility +0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport +0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA +0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler +0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment +0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay +0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity +0x000000028a8ce000 /usr/lib/libTLE.dylib +0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper +0x00000001e488a000 /usr/lib/libedit.3.dylib +0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL +0x00000001be335000 /usr/lib/libncurses.5.4.dylib +0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji +0x000000018dd85000 /usr/lib/libCRFSuite.dylib +0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP +0x000000019b578000 /usr/lib/libmecab.dylib +0x000000019c485000 /usr/lib/libThaiTokenizer.dylib +0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay +0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI +0x00000001a08c1000 /usr/lib/libcups.2.dylib +0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos +0x00000001a05b8000 /usr/lib/libresolv.9.dylib +0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal +0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib +0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth +0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities +0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib +0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib +0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib +0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib +0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib +0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib +0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib +0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib +0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth +0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport +0x000000019b542000 /usr/lib/libCheckFix.dylib +0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities +0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary +0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore +0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices +0x000000019d93a000 /usr/lib/libxslt.1.dylib +0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement +0x00000001a8999000 /usr/lib/libcurl.4.dylib +0x000000028ab07000 /usr/lib/libcrypto.46.dylib +0x000000028b727000 /usr/lib/libssl.48.dylib +0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP +0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent +0x00000001a05d5000 /usr/lib/libsasl2.2.dylib +0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib +0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib +0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib +0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib +0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial +0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib +0x000000028a8cb000 /usr/lib/libSpatial.dylib +0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities +0x0000000107d58000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib +0x00000001021e4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib +0x0000000102214000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib +0x0000000102264000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib +0x00000001023ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib +0x0000000102414000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib +0x000000010775c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib +0x0000000107780000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib +0x00000001023d8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib +0x0000000107824000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib +0x0000000107838000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib +0x000000010784c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib +0x0000000147dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib +0x000000010789c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15423878368099207039.dylib +0x0000000107948000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-afedc8a6-4493-4c3c-aed3-c280414ad816-libsnappyjava.dylib + + +VM Arguments: +jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false +java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. +java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 12 {product} {ergonomic} + uint ConcGCThreads = 5 {product} {ergonomic} + uint G1ConcRefinementThreads = 20 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + bool IgnoreUnrecognizedVMOptions = true {product} {command line} + size_t InitialHeapSize = 1610612736 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4294967296 {product} {command line} + size_t MaxNewSize = 2575302656 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} + intx ThreadStackSize = 4096 {pd product} {command line} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + bool UseNUMA = false {product} {ergonomic} + bool UseNUMAInterleaving = false {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home +CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar +PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin +SHELL=/bin/zsh +LANG=en_US.UTF-8 +TERM=xterm-256color +TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked + + +Periodic native trim disabled + + +--------------- S Y S T E M --------------- + +OS: +uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 +OS uptime: 4 days 1:42 hours +rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity +load average: 8.23 7.63 5.76 + +CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse +machdep.cpu.brand_string:Apple M3 Ultra +hw.cachelinesize:128 +hw.l1icachesize:131072 +hw.l1dcachesize:65536 +hw.l2cachesize:4194304 + +Memory: 16k page, physical 100663296k(3080768k free), swap 2097152k(1070592k free) + +vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) + +END. diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 1a2a2260a1..9dad73fef9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -72,8 +72,15 @@ class CometNativeShuffleWriter[K, V]( val tempDataFilePath = Paths.get(tempDataFilename) val tempIndexFilePath = Paths.get(tempIndexFilename) + // Detect if input comes from a native plan (CometExecIterator) + val nativeIter: Option[CometExecIterator] = inputs match { + case swi: CometShuffleWriterInputIterator => swi.nativeIterator + case _ => None + } + val useHandleMode = nativeIter.isDefined + // Call native shuffle write - val nativePlan = getNativePlan(tempDataFilename, tempIndexFilename) + val nativePlan = getNativePlan(tempDataFilename, tempIndexFilename, useHandleMode) val detailedMetrics = Seq( "elapsed_compute", @@ -93,12 +100,6 @@ class CometNativeShuffleWriter[K, V]( metrics.filterKeys(detailedMetrics.contains) val nativeMetrics = CometMetricNode(nativeSQLMetrics) - // Detect if input comes from a native plan (CometExecIterator) - val nativeIter: Option[CometExecIterator] = inputs match { - case swi: CometShuffleWriterInputIterator => swi.nativeIterator - case _ => None - } - val cometIter = nativeIter match { case Some(childIter) => // Stash mode: child plan stashes batches, shuffle writer retrieves via handles @@ -181,8 +182,12 @@ class CometNativeShuffleWriter[K, V]( case _ => false } - private def getNativePlan(dataFile: String, indexFile: String): Operator = { - val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("ShuffleWriterInput") + private def getNativePlan( + dataFile: String, + indexFile: String, + useHandleMode: Boolean = false): Operator = { + val scanSource = if (useHandleMode) "ShuffleWriterInputHandle" else "ShuffleWriterInput" + val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource(scanSource) val opBuilder = OperatorOuterClass.Operator.newBuilder() val scanTypes = outputAttributes.flatten { attr => From 3af0e9a87508c0ab59b61723c430fa4b3b8ee1de Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:15:18 -0600 Subject: [PATCH 13/21] chore: remove JVM crash log files --- spark/hs_err_pid58786.log | 1611 ---------------- spark/hs_err_pid58919.log | 1602 ---------------- spark/hs_err_pid59058.log | 1568 --------------- spark/hs_err_pid59157.log | 3785 ------------------------------------- spark/hs_err_pid59323.log | 1626 ---------------- spark/hs_err_pid59395.log | 1886 ------------------ 6 files changed, 12078 deletions(-) delete mode 100644 spark/hs_err_pid58786.log delete mode 100644 spark/hs_err_pid58919.log delete mode 100644 spark/hs_err_pid59058.log delete mode 100644 spark/hs_err_pid59157.log delete mode 100644 spark/hs_err_pid59323.log delete mode 100644 spark/hs_err_pid59395.log diff --git a/spark/hs_err_pid58786.log b/spark/hs_err_pid58786.log deleted file mode 100644 index 26eadbb201..0000000000 --- a/spark/hs_err_pid58786.log +++ /dev/null @@ -1,1611 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x0000000106d89f64, pid=58786, tid=67331 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 08:56:28 2026 MDT elapsed time: 21.994476 seconds (0d 0h 0m 21s) - ---------------- T H R E A D --------------- - -Current thread (0x0000000755abea00): JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=67331, stack(0x0000000401834000,0x0000000401c37000)] - -Stack: [0x0000000401834000,0x0000000401c37000], sp=0x0000000401c1e120, free space=4008k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -C [libcomet-14714718128898053499.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 -C [libcomet-14714718128898053499.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 -C [libcomet-14714718128898053499.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c -C [libcomet-14714718128898053499.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 -C [libcomet-14714718128898053499.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 -C [libcomet-14714718128898053499.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 -C [libcomet-14714718128898053499.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 -C [libcomet-14714718128898053499.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-14714718128898053499.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-14714718128898053499.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-14714718128898053499.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-14714718128898053499.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-14714718128898053499.dylib+0x106628] __rust_try+0x20 -C [libcomet-14714718128898053499.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-14714718128898053499.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-14714718128898053499.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-14714718128898053499.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3960+0x000000f8021c6e40.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3955+0x000000f8021c5a70.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2521+0x000000f801df1e28.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3960+0x000000f8021c6e40.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3955+0x000000f8021c5a70.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2521+0x000000f801df1e28.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x656c666675685343 - -Registers: - x0=0x000000f8021af168 x1=0x0000000000000010 x2=0x000000075c4e85e0 x3=0x0000000401c23298 - x4=0x0000000000000238 x5=0x0000000000000018 x6=0x000000075c77c8a0 x7=0xfffff0003ffff800 - x8=0x00000000021af168 x9=0x0000000000000000 x10=0x000000f800000000 x11=0x0000000753584040 -x12=0x000000075c06abc0 x13=0x0000000000000001 x14=0x0000000106479a68 x15=0x0000000753584000 -x16=0x000000018c8009d0 x17=0x00000007529a0500 x18=0x0000000000000000 x19=0x0000000755abea00 -x20=0x0000000000000001 x21=0x00000001014ee248 x22=0x656c66667568532f x23=0x00000001074e9088 -x24=0x0000000000000000 x25=0x00000001074edf38 x26=0x0000000134fda0a8 x27=0x00000003033a7dc0 -x28=0x0000000755abea00 fp=0x0000000401c1e160 lr=0x0000000106d89f38 sp=0x0000000401c1e120 -pc=0x0000000106d89f64 cpsr=0x0000000040001000 - -Register to memory mapping: - - x0=0x000000f8021af168 is a pointer to class: -org.apache.comet.CometHandleBatchIterator {0x000000f8021af168} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x0000000135051340) - - method ordering: Array(0x000000f800913f08) - - local interfaces: Array(0x000000f8005d4fb8) - - trans. interfaces: Array(0x000000f8005d4fb8) - - constants: constant pool [62]/operands[4] {0x00000001350510b0} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000135051590 - - class loader data: loader data: 0x000000075c020fa0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x000000030330ac58} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x000000f8005d4f48) - - nest members: Array(0x000000f8005d4f48) - - permitted subclasses: Array(0x000000f8005d4f48) - - java mirror: a 'java/lang/Class'{0x0000000313c484f8} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x000000f8021af338) - - itable length 2 (start addr: 0x000000f8021af368) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x1=0x0000000000000010 is an unknown value - x2=0x000000075c4e85e0 points into unknown readable memory: 0x000000032a664cf0 | f0 4c 66 2a 03 00 00 00 - x3=0x0000000401c23298 is pointing into the stack for thread: 0x0000000755abea00 - x4=0x0000000000000238 is an unknown value - x5=0x0000000000000018 is an unknown value - x6=0x000000075c77c8a0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=35320168 is a compressed pointer to class: 0x000000f8021af168 -org.apache.comet.CometHandleBatchIterator {0x000000f8021af168} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x0000000135051340) - - method ordering: Array(0x000000f800913f08) - - local interfaces: Array(0x000000f8005d4fb8) - - trans. interfaces: Array(0x000000f8005d4fb8) - - constants: constant pool [62]/operands[4] {0x00000001350510b0} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000135051590 - - class loader data: loader data: 0x000000075c020fa0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x000000030330ac58} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x000000f8005d4f48) - - nest members: Array(0x000000f8005d4f48) - - permitted subclasses: Array(0x000000f8005d4f48) - - java mirror: a 'java/lang/Class'{0x0000000313c484f8} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x000000f8021af338) - - itable length 2 (start addr: 0x000000f8021af368) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x9=0x0 is NULL -x10=0x000000f800000000 is pointing into metadata -x11=0x0000000753584040 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -x12=0x000000075c06abc0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x13=0x0000000000000001 is an unknown value -x14=0x0000000106479a68 points into unknown readable memory: 0x02d626b000c7e800 | 00 e8 c7 00 b0 26 d6 02 -x15=0x0000000753584000 points into unknown readable memory: 0x4a191953546ffd27 | 27 fd 6f 54 53 19 19 4a -x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 -x17=0x00000007529a0500 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -x18=0x0 is NULL -x19=0x0000000755abea00 is a thread -x20=0x0000000000000001 is an unknown value -x21=0x00000001014ee248 is a global jni handle -x22=0x656c66667568532f is an unknown value -x23=0x00000001074e9088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 -x24=0x0 is NULL -x25=0x00000001074edf38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 -x26=0x0000000134fda0a8 is pointing into metadata -x27=0x00000003033a7dc0 is an oop: [Z -{0x00000003033a7dc0} - klass: {type array bool} - - length: 9 -x28=0x0000000755abea00 is a thread - - -Top of Stack: (sp=0x0000000401c1e120) -0x0000000401c1e120: 0000000134fda0a8 000000030331a520 -0x0000000401c1e130: 0000000401c35818 0000000303319f00 -0x0000000401c1e140: 0000000000000000 0000000107537f90 -0x0000000401c1e150: 0000000401c35738 000000011687372c -0x0000000401c1e160: 0000000401c1e5f0 000000014c74ce4c -0x0000000401c1e170: 0000000000000000 000000014f66e880 -0x0000000401c1e180: 0000000401c1e190 000000014c786d54 -0x0000000401c1e190: 000000075c4e85e0 00000001014ee248 -0x0000000401c1e1a0: 0000000106d89e08 00000007586547b0 -0x0000000401c1e1b0: 00000001074f2ad0 00000001074f2ad0 -0x0000000401c1e1c0: 0000000755abecb0 0000000755abecb0 -0x0000000401c1e1d0: 000000014f67ae90 000000004c74ee14 -0x0000000401c1e1e0: 0000000000000000 0000000401c1e270 -0x0000000401c1e1f0: 0000000755abecb0 000000014f67ae90 -0x0000000401c1e200: 0000000055abea00 0000000401c1e860 -0x0000000401c1e210: 0000000401c1ea00 8000000000000020 -0x0000000401c1e220: 000000014f67ae90 00000007586547b0 -0x0000000401c1e230: 000000014f67ae90 8000000000000020 -0x0000000401c1e240: 000000014f67ae90 0000000755abecb0 -0x0000000401c1e250: 0000000755abecb0 0000000000000002 -0x0000000401c1e260: 00000007529a0500 0000000000000000 -0x0000000401c1e270: 0000000755abecb0 0000000000000002 -0x0000000401c1e280: 00000007529a0500 8000000000000020 -0x0000000401c1e290: 000000014f67ae90 00000007529a05d0 -0x0000000401c1e2a0: 0000000401c1e2e0 000000014c7afc28 -0x0000000401c1e2b0: 0000000000000000 0000000000000000 -0x0000000401c1e2c0: 0000000755abecb0 000000014f66e888 -0x0000000401c1e2d0: 0000000000000000 0000000755abecb0 -0x0000000401c1e2e0: 0000000755abecb0 0000000000000002 -0x0000000401c1e2f0: 0000000000000008 000000014f66e888 -0x0000000401c1e300: 0000000401c1e340 000000014c787d8c -0x0000000401c1e310: 0000000401c1e3f8 0000000755abecb0 - -Instructions: (pc=0x0000000106d89f64) -0x0000000106d89e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 -0x0000000106d89e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 -0x0000000106d89e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 -0x0000000106d89e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 -0x0000000106d89ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 -0x0000000106d89eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 -0x0000000106d89ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 -0x0000000106d89ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 -0x0000000106d89ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 -0x0000000106d89ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a -0x0000000106d89f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 -0x0000000106d89f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa -0x0000000106d89f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 -0x0000000106d89f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 -0x0000000106d89f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 -0x0000000106d89f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b -0x0000000106d89f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 -0x0000000106d89f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 -0x0000000106d89f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 -0x0000000106d89f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb -0x0000000106d89fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 -0x0000000106d89fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 -0x0000000106d89fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 -0x0000000106d89fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 -0x0000000106d89fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 -0x0000000106d89ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 -0x0000000106d8a004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b -0x0000000106d8a014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 -0x0000000106d8a024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 -0x0000000106d8a034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 -0x0000000106d8a044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 -0x0000000106d8a054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x0000000134fda0a8 is pointing into metadata -stack at sp + 1 slots: 0x000000030331a520 is an oop: [Z -{0x000000030331a520} - klass: {type array bool} - - length: 264 -stack at sp + 2 slots: 0x0000000401c35818 is pointing into the stack for thread: 0x0000000755abea00 -stack at sp + 3 slots: 0x0000000303319f00 is an oop: [Z -{0x0000000303319f00} - klass: {type array bool} - - length: 187 -stack at sp + 4 slots: 0x0 is NULL -stack at sp + 5 slots: 0x0000000107537f90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x00000001068cc000 -stack at sp + 6 slots: 0x0000000401c35738 is pointing into the stack for thread: 0x0000000755abea00 -stack at sp + 7 slots: 0x000000011687372c is at code_begin+44 in an Interpreter codelet -result handlers for native calls [0x0000000116873700, 0x0000000116873780] 128 bytes - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x0000000764145140, length=77, elements={ -0x00000001014d1e20, 0x00000001014e6da0, 0x0000000758d80000, 0x0000000758d80c00, -0x0000000758d81200, 0x0000000758d81800, 0x0000000758d81e00, 0x0000000758d82400, -0x0000000758d82a00, 0x0000000758d83000, 0x0000000758e9e400, 0x0000000755245800, -0x000000075589d200, 0x000000075589e400, 0x000000075589ea00, 0x000000075589f000, -0x000000075589f600, 0x000000075586c600, 0x00000007552a7600, 0x0000000756ff0000, -0x0000000755246400, 0x0000000755244c00, 0x0000000758350000, 0x0000000758350600, -0x0000000758350c00, 0x0000000758351200, 0x0000000758353600, 0x0000000758375800, -0x0000000758375e00, 0x0000000758377000, 0x0000000758377600, 0x000000075839c600, -0x000000075839cc00, 0x000000075839d200, 0x000000075839de00, 0x000000075839e400, -0x000000075839f600, 0x00000007583c0000, 0x00000007583c0c00, 0x00000007583c1800, -0x00000007583c1e00, 0x00000007583c2400, 0x00000007583c2a00, 0x0000000754843000, -0x000000075488cc00, 0x000000075488d800, 0x0000000754970600, 0x0000000754670000, -0x0000000755abd200, 0x0000000755abd800, 0x0000000755abde00, 0x0000000755abe400, -0x0000000755abea00, 0x0000000755acaa00, 0x0000000755abf600, 0x0000000754789e00, -0x00000007575af600, 0x0000000754d86a00, 0x00000007544cb000, 0x0000000755ac8000, -0x0000000755ac8600, 0x0000000752906a00, 0x0000000752907000, 0x0000000752907600, -0x0000000752974000, 0x0000000752974600, 0x0000000752b5d200, 0x0000000752b5d800, -0x0000000752b5de00, 0x0000000752b5e400, 0x0000000752b5ea00, 0x0000000758e9f600, -0x0000000755abf000, 0x0000000755abcc00, 0x0000000755acb000, 0x0000000756ff3000, -0x00000007535b4000 -} - -Java Threads: ( => current thread ) - 0x00000001014d1e20 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5891, stack(0x000000016f0f0000,0x000000016f4f3000)] - 0x00000001014e6da0 JavaThread "Reference Handler" daemon [_thread_blocked, id=22787, stack(0x0000000170144000,0x0000000170547000)] - 0x0000000758d80000 JavaThread "Finalizer" daemon [_thread_blocked, id=30723, stack(0x0000000170550000,0x0000000170953000)] - 0x0000000758d80c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=29955, stack(0x0000000170a74000,0x0000000170e77000)] - 0x0000000758d81200 JavaThread "Service Thread" daemon [_thread_blocked, id=29443, stack(0x0000000170e80000,0x0000000171283000)] - 0x0000000758d81800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=25859, stack(0x000000017128c000,0x000000017168f000)] - 0x0000000758d81e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26371, stack(0x0000000171698000,0x000000017189b000)] - 0x0000000758d82400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=29187, stack(0x00000001718a4000,0x0000000171aa7000)] - 0x0000000758d82a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=28675, stack(0x0000000171ab0000,0x0000000171eb3000)] - 0x0000000758d83000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=26883, stack(0x0000000171ebc000,0x00000001722bf000)] - 0x0000000758e9e400 JavaThread "Notification Thread" daemon [_thread_blocked, id=33539, stack(0x0000000172d04000,0x0000000173107000)] - 0x0000000755245800 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45315, stack(0x0000000176024000,0x0000000176427000)] - 0x000000075589d200 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=62723, stack(0x0000000176e6c000,0x000000017726f000)] - 0x000000075589e400 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=62467, stack(0x0000000177278000,0x000000017767b000)] - 0x000000075589ea00 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=47619, stack(0x0000000177684000,0x0000000177a87000)] - 0x000000075589f000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=47875, stack(0x0000000177a90000,0x0000000177e93000)] - 0x000000075589f600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=61955, stack(0x0000000177e9c000,0x000000017829f000)] - 0x000000075586c600 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=48387, stack(0x00000001782a8000,0x00000001786ab000)] - 0x00000007552a7600 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=48899, stack(0x00000001786b4000,0x0000000178ab7000)] - 0x0000000756ff0000 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=61187, stack(0x0000000178ac0000,0x0000000178ec3000)] - 0x0000000755246400 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=60675, stack(0x0000000178ecc000,0x00000001792cf000)] - 0x0000000755244c00 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=60419, stack(0x00000001792d8000,0x00000001796db000)] - 0x0000000758350000 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=59907, stack(0x00000001796e4000,0x0000000179ae7000)] - 0x0000000758350600 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=59651, stack(0x0000000179af0000,0x0000000179ef3000)] - 0x0000000758350c00 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49923, stack(0x0000000179efc000,0x000000017a2ff000)] - 0x0000000758351200 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=50435, stack(0x000000017a308000,0x000000017a70b000)] - 0x0000000758353600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50691, stack(0x000000017a714000,0x000000017ab17000)] - 0x0000000758375800 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=59139, stack(0x000000017ab20000,0x000000017af23000)] - 0x0000000758375e00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=58883, stack(0x000000017af2c000,0x000000017b32f000)] - 0x0000000758377000 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58371, stack(0x000000017b338000,0x000000017b73b000)] - 0x0000000758377600 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=58115, stack(0x000000017b744000,0x000000017bb47000)] - 0x000000075839c600 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=57859, stack(0x000000017bb50000,0x000000017bf53000)] - 0x000000075839cc00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57347, stack(0x000000017bf5c000,0x000000017c35f000)] - 0x000000075839d200 JavaThread "task-abort-timer" daemon [_thread_blocked, id=57091, stack(0x000000017c368000,0x000000017c76b000)] - 0x000000075839de00 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=56579, stack(0x000000017c774000,0x000000017cb77000)] - 0x000000075839e400 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52483, stack(0x000000017cb80000,0x000000017cf83000)] - 0x000000075839f600 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=52739, stack(0x000000017cf8c000,0x000000017d38f000)] - 0x00000007583c0000 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=56067, stack(0x000000017d398000,0x000000017d79b000)] - 0x00000007583c0c00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=53507, stack(0x000000017d7a4000,0x000000017dba7000)] - 0x00000007583c1800 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=55811, stack(0x000000017dbb0000,0x000000017dfb3000)] - 0x00000007583c1e00 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54275, stack(0x000000017dfbc000,0x000000017e3bf000)] - 0x00000007583c2400 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=55555, stack(0x000000017e3c8000,0x000000017e7cb000)] - 0x00000007583c2a00 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55299, stack(0x000000017e7d4000,0x000000017ebd7000)] - 0x0000000754843000 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=65539, stack(0x000000017ebe0000,0x000000017efe3000)] - 0x000000075488cc00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=87043, stack(0x000000017efec000,0x000000017f3ef000)] - 0x000000075488d800 JavaThread "Timer-0" [_thread_blocked, id=86531, stack(0x000000017f3f8000,0x000000017f7fb000)] - 0x0000000754970600 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=66051, stack(0x000000017f804000,0x000000017fc07000)] - 0x0000000754670000 JavaThread "process reaper" daemon [_thread_blocked, id=86027, stack(0x000000017fc10000,0x000000017fc47000)] - 0x0000000755abd200 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x0000000400804000,0x0000000400c07000)] - 0x0000000755abd800 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=41523, stack(0x0000000400c10000,0x0000000401013000)] - 0x0000000755abde00 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=30223, stack(0x000000040101c000,0x000000040141f000)] - 0x0000000755abe400 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=85763, stack(0x0000000401428000,0x000000040182b000)] -=>0x0000000755abea00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=67331, stack(0x0000000401834000,0x0000000401c37000)] - 0x0000000755acaa00 JavaThread "process reaper" daemon [_thread_blocked, id=68099, stack(0x00000001709e8000,0x0000000170a1f000)] - 0x0000000755abf600 JavaThread "process reaper" daemon [_thread_blocked, id=83715, stack(0x0000000170a28000,0x0000000170a5f000)] - 0x0000000754789e00 JavaThread "process reaper" daemon [_thread_blocked, id=83459, stack(0x000000017fe5c000,0x000000017fe93000)] - 0x00000007575af600 JavaThread "process reaper" daemon [_thread_blocked, id=69379, stack(0x000000017fe9c000,0x000000017fed3000)] - 0x0000000754d86a00 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=42771, stack(0x000000017331c000,0x000000017371f000)] - 0x00000007544cb000 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=42051, stack(0x0000000401c40000,0x0000000402043000)] - 0x0000000755ac8000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=45839, stack(0x000000040204c000,0x000000040244f000)] - 0x0000000755ac8600 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=67855, stack(0x0000000402458000,0x000000040285b000)] - 0x0000000752906a00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=83267, stack(0x0000000402864000,0x0000000402c67000)] - 0x0000000752907000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=83991, stack(0x0000000402c70000,0x0000000403073000)] - 0x0000000752907600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=70171, stack(0x000000040307c000,0x000000040347f000)] - 0x0000000752974000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=82459, stack(0x0000000403488000,0x000000040388b000)] - 0x0000000752974600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=82179, stack(0x0000000403894000,0x0000000403c97000)] - 0x0000000752b5d200 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=28179, stack(0x00000001722c8000,0x00000001724cb000)] - 0x0000000752b5d800 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=79887, stack(0x00000001724d4000,0x00000001726d7000)] - 0x0000000752b5de00 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=80403, stack(0x00000001726e0000,0x00000001728e3000)] - 0x0000000752b5e400 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=80139, stack(0x00000001728ec000,0x0000000172aef000)] - 0x0000000752b5ea00 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=33331, stack(0x0000000172af8000,0x0000000172cfb000)] - 0x0000000758e9f600 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=43071, stack(0x00000004044d0000,0x00000004048d3000)] - 0x0000000755abf000 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=43327, stack(0x00000004048dc000,0x0000000404cdf000)] - 0x0000000755abcc00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=84771, stack(0x0000000404d00000,0x0000000405103000)] - 0x0000000755acb000 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=84499, stack(0x000000040510c000,0x000000040550f000)] - 0x0000000756ff3000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=83011, stack(0x0000000405518000,0x000000040591b000)] - 0x00000007535b4000 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=72707, stack(0x0000000405924000,0x0000000405d27000)] - -Other Threads: - 0x00000001014e6120 VMThread "VM Thread" [stack: 0x000000016ff38000,0x000000017013b000] [id=20227] - 0x0000000758604600 WatcherThread [stack: 0x0000000173110000,0x0000000173313000] [id=33795] - 0x00000001014de240 GCTaskThread "GC Thread#0" [stack: 0x000000016f4fc000,0x000000016f6ff000] [id=14339] - 0x0000000756f5d200 GCTaskThread "GC Thread#1" [stack: 0x0000000173940000,0x0000000173b43000] [id=41219] - 0x0000000756f5d500 GCTaskThread "GC Thread#2" [stack: 0x0000000173b4c000,0x0000000173d4f000] [id=34819] - 0x0000000756f5d800 GCTaskThread "GC Thread#3" [stack: 0x0000000173d58000,0x0000000173f5b000] [id=35075] - 0x0000000756f5db00 GCTaskThread "GC Thread#4" [stack: 0x0000000173f64000,0x0000000174167000] [id=35587] - 0x0000000756f5de00 GCTaskThread "GC Thread#5" [stack: 0x0000000174170000,0x0000000174373000] [id=35843] - 0x0000000756f5e100 GCTaskThread "GC Thread#6" [stack: 0x000000017437c000,0x000000017457f000] [id=39939] - 0x0000000756f5e400 GCTaskThread "GC Thread#7" [stack: 0x0000000174588000,0x000000017478b000] [id=36099] - 0x0000000756f5e700 GCTaskThread "GC Thread#8" [stack: 0x0000000174794000,0x0000000174997000] [id=36611] - 0x0000000756f5ea00 GCTaskThread "GC Thread#9" [stack: 0x00000001749a0000,0x0000000174ba3000] [id=36867] - 0x0000000756f5ed00 GCTaskThread "GC Thread#10" [stack: 0x0000000174bac000,0x0000000174daf000] [id=38915] - 0x0000000756f5f000 GCTaskThread "GC Thread#11" [stack: 0x0000000174db8000,0x0000000174fbb000] [id=38659] - 0x0000000756f5f300 GCTaskThread "GC Thread#12" [stack: 0x0000000174fc4000,0x00000001751c7000] [id=38403] - 0x0000000756f5f600 GCTaskThread "GC Thread#13" [stack: 0x00000001751d0000,0x00000001753d3000] [id=37635] - 0x0000000756f5f900 GCTaskThread "GC Thread#14" [stack: 0x00000001753dc000,0x00000001755df000] [id=43523] - 0x0000000756f5fc00 GCTaskThread "GC Thread#15" [stack: 0x00000001755e8000,0x00000001757eb000] [id=65283] - 0x00000007567fc000 GCTaskThread "GC Thread#16" [stack: 0x00000001757f4000,0x00000001759f7000] [id=65027] - 0x00000007567fc300 GCTaskThread "GC Thread#17" [stack: 0x0000000175a00000,0x0000000175c03000] [id=64771] - 0x00000007567fc600 GCTaskThread "GC Thread#18" [stack: 0x0000000175c0c000,0x0000000175e0f000] [id=44547] - 0x00000007567fc900 GCTaskThread "GC Thread#19" [stack: 0x0000000175e18000,0x000000017601b000] [id=44803] - 0x00000001014def60 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016f708000,0x000000016f90b000] [id=12547] - 0x00000001014df860 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016f914000,0x000000016fb17000] [id=13059] - 0x000000075534d800 ConcurrentGCThread "G1 Conc#1" [stack: 0x000000017663c000,0x000000017683f000] [id=63747] - 0x000000075534de00 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000176848000,0x0000000176a4b000] [id=63235] - 0x000000075534e100 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000176a54000,0x0000000176c57000] [id=46083] - 0x000000075534e400 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000176c60000,0x0000000176e63000] [id=46595] - 0x00000001014e1b20 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016fb20000,0x000000016fd23000] [id=16643] - 0x00000001014e2420 ConcurrentGCThread "G1 Service" [stack: 0x000000016fd2c000,0x000000016ff2f000] [id=21507] - -Threads with active compile tasks: - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x0000000300800000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x000000f800000000-0x000000f800bc0000-0x000000f800bc0000), size 12320768, SharedBaseAddress: 0x000000f800000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x000000f801000000-0x000000f841000000, reserved size: 1073741824 -Narrow klass base: 0x000000f800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 348160K, used 181790K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 15 young (30720K), 4 survivors (8192K) - Metaspace used 151515K, committed 152832K, reserved 1245184K - class space used 18221K, committed 18816K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000300800000, 0x0000000300a00000, 0x0000000300a00000|100%|HS| |TAMS 0x0000000300a00000, 0x0000000300a00000| Complete -| 1|0x0000000300a00000, 0x0000000300c00000, 0x0000000300c00000|100%|HS| |TAMS 0x0000000300c00000, 0x0000000300c00000| Complete -| 2|0x0000000300c00000, 0x0000000300e00000, 0x0000000300e00000|100%|HC| |TAMS 0x0000000300e00000, 0x0000000300e00000| Complete -| 3|0x0000000300e00000, 0x0000000301000000, 0x0000000301000000|100%|HS| |TAMS 0x0000000301000000, 0x0000000301000000| Complete -| 4|0x0000000301000000, 0x0000000301200000, 0x0000000301200000|100%|HC| |TAMS 0x0000000301200000, 0x0000000301200000| Complete -| 5|0x0000000301200000, 0x0000000301400000, 0x0000000301400000|100%|HC| |TAMS 0x0000000301400000, 0x0000000301400000| Complete -| 6|0x0000000301400000, 0x0000000301600000, 0x0000000301600000|100%|HC| |TAMS 0x0000000301600000, 0x0000000301600000| Complete -| 7|0x0000000301600000, 0x0000000301800000, 0x0000000301800000|100%|HC| |TAMS 0x0000000301800000, 0x0000000301800000| Complete -| 8|0x0000000301800000, 0x0000000301a00000, 0x0000000301a00000|100%|HC| |TAMS 0x0000000301a00000, 0x0000000301a00000| Complete -| 9|0x0000000301a00000, 0x0000000301c00000, 0x0000000301c00000|100%|HC| |TAMS 0x0000000301c00000, 0x0000000301c00000| Complete -| 10|0x0000000301c00000, 0x0000000301e00000, 0x0000000301e00000|100%|HC| |TAMS 0x0000000301e00000, 0x0000000301e00000| Complete -| 11|0x0000000301e00000, 0x0000000302000000, 0x0000000302000000|100%|HS| |TAMS 0x0000000302000000, 0x0000000302000000| Complete -| 12|0x0000000302000000, 0x0000000302200000, 0x0000000302200000|100%|HS| |TAMS 0x0000000302200000, 0x0000000302200000| Complete -| 13|0x0000000302200000, 0x0000000302400000, 0x0000000302400000|100%|HC| |TAMS 0x0000000302400000, 0x0000000302400000| Complete -| 14|0x0000000302400000, 0x0000000302600000, 0x0000000302600000|100%|HS| |TAMS 0x0000000302600000, 0x0000000302600000| Complete -| 15|0x0000000302600000, 0x0000000302800000, 0x0000000302800000|100%|HC| |TAMS 0x0000000302800000, 0x0000000302800000| Complete -| 16|0x0000000302800000, 0x0000000302a00000, 0x0000000302a00000|100%|HS| |TAMS 0x0000000302a00000, 0x0000000302a00000| Complete -| 17|0x0000000302a00000, 0x0000000302c00000, 0x0000000302c00000|100%|HS| |TAMS 0x0000000302c00000, 0x0000000302c00000| Complete -| 18|0x0000000302c00000, 0x0000000302e00000, 0x0000000302e00000|100%|HC| |TAMS 0x0000000302e00000, 0x0000000302e00000| Complete -| 19|0x0000000302e00000, 0x0000000303000000, 0x0000000303000000|100%| O| |TAMS 0x0000000303000000, 0x0000000303000000| Untracked -| 20|0x0000000303000000, 0x0000000303200000, 0x0000000303200000|100%| O| |TAMS 0x0000000303200000, 0x0000000303200000| Untracked -| 21|0x0000000303200000, 0x0000000303400000, 0x0000000303400000|100%| O| |TAMS 0x0000000303400000, 0x0000000303400000| Untracked -| 22|0x0000000303400000, 0x0000000303600000, 0x0000000303600000|100%| O| |TAMS 0x0000000303600000, 0x0000000303600000| Untracked -| 23|0x0000000303600000, 0x0000000303800000, 0x0000000303800000|100%| O| |TAMS 0x0000000303800000, 0x0000000303800000| Untracked -| 24|0x0000000303800000, 0x0000000303a00000, 0x0000000303a00000|100%| O| |TAMS 0x0000000303a00000, 0x0000000303a00000| Untracked -| 25|0x0000000303a00000, 0x0000000303c00000, 0x0000000303c00000|100%| O| |TAMS 0x0000000303c00000, 0x0000000303c00000| Untracked -| 26|0x0000000303c00000, 0x0000000303d9a600, 0x0000000303e00000| 80%| O| |TAMS 0x0000000303d9a600, 0x0000000303d9a600| Untracked -| 27|0x0000000303e00000, 0x0000000304000000, 0x0000000304000000|100%| O| |TAMS 0x0000000304000000, 0x0000000304000000| Untracked -| 28|0x0000000304000000, 0x0000000304163800, 0x0000000304200000| 69%| O| |TAMS 0x0000000304163800, 0x0000000304163800| Untracked -| 29|0x0000000304200000, 0x0000000304400000, 0x0000000304400000|100%| O| |TAMS 0x0000000304400000, 0x0000000304400000| Untracked -| 30|0x0000000304400000, 0x0000000304600000, 0x0000000304600000|100%| O| |TAMS 0x0000000304600000, 0x0000000304600000| Untracked -| 31|0x0000000304600000, 0x0000000304800000, 0x0000000304800000|100%| O| |TAMS 0x0000000304800000, 0x0000000304800000| Untracked -| 32|0x0000000304800000, 0x0000000304a00000, 0x0000000304a00000|100%| O| |TAMS 0x0000000304a00000, 0x0000000304a00000| Untracked -| 33|0x0000000304a00000, 0x0000000304c00000, 0x0000000304c00000|100%| O| |TAMS 0x0000000304c00000, 0x0000000304c00000| Untracked -| 34|0x0000000304c00000, 0x0000000304e00000, 0x0000000304e00000|100%| O| |TAMS 0x0000000304e00000, 0x0000000304e00000| Untracked -| 35|0x0000000304e00000, 0x0000000305000000, 0x0000000305000000|100%| O| |TAMS 0x0000000305000000, 0x0000000305000000| Untracked -| 36|0x0000000305000000, 0x0000000305200000, 0x0000000305200000|100%| O| |TAMS 0x0000000305200000, 0x0000000305200000| Untracked -| 37|0x0000000305200000, 0x0000000305400000, 0x0000000305400000|100%| O| |TAMS 0x0000000305400000, 0x0000000305400000| Untracked -| 38|0x0000000305400000, 0x0000000305600000, 0x0000000305600000|100%| O| |TAMS 0x0000000305600000, 0x0000000305600000| Untracked -| 39|0x0000000305600000, 0x0000000305800000, 0x0000000305800000|100%| O| |TAMS 0x0000000305800000, 0x0000000305800000| Untracked -| 40|0x0000000305800000, 0x0000000305a00000, 0x0000000305a00000|100%| O| |TAMS 0x0000000305a00000, 0x0000000305a00000| Untracked -| 41|0x0000000305a00000, 0x0000000305c00000, 0x0000000305c00000|100%| O| |TAMS 0x0000000305c00000, 0x0000000305c00000| Untracked -| 42|0x0000000305c00000, 0x0000000305d80a00, 0x0000000305e00000| 75%| O| |TAMS 0x0000000305d80a00, 0x0000000305d80a00| Untracked -| 43|0x0000000305e00000, 0x0000000306000000, 0x0000000306000000|100%|HS| |TAMS 0x0000000305e00000, 0x0000000306000000| Complete -| 44|0x0000000306000000, 0x0000000306200000, 0x0000000306200000|100%|HS| |TAMS 0x0000000306000000, 0x0000000306200000| Complete -| 45|0x0000000306200000, 0x0000000306400000, 0x0000000306400000|100%|HS| |TAMS 0x0000000306200000, 0x0000000306400000| Complete -| 46|0x0000000306400000, 0x0000000306600000, 0x0000000306600000|100%|HS| |TAMS 0x0000000306400000, 0x0000000306600000| Complete -| 47|0x0000000306600000, 0x0000000306800000, 0x0000000306800000|100%|HS| |TAMS 0x0000000306600000, 0x0000000306800000| Complete -| 48|0x0000000306800000, 0x0000000306a00000, 0x0000000306a00000|100%|HS| |TAMS 0x0000000306800000, 0x0000000306a00000| Complete -| 49|0x0000000306a00000, 0x0000000306c00000, 0x0000000306c00000|100%|HS| |TAMS 0x0000000306a00000, 0x0000000306c00000| Complete -| 50|0x0000000306c00000, 0x0000000306e00000, 0x0000000306e00000|100%|HS| |TAMS 0x0000000306c00000, 0x0000000306e00000| Complete -| 51|0x0000000306e00000, 0x0000000307000000, 0x0000000307000000|100%|HS| |TAMS 0x0000000306e00000, 0x0000000307000000| Complete -| 52|0x0000000307000000, 0x0000000307200000, 0x0000000307200000|100%|HS| |TAMS 0x0000000307000000, 0x0000000307200000| Complete -| 53|0x0000000307200000, 0x0000000307400000, 0x0000000307400000|100%| O| |TAMS 0x0000000307200000, 0x0000000307400000| Untracked -| 54|0x0000000307400000, 0x0000000307600000, 0x0000000307600000|100%| O| |TAMS 0x0000000307400000, 0x0000000307600000| Untracked -| 55|0x0000000307600000, 0x0000000307800000, 0x0000000307800000|100%| O| |TAMS 0x0000000307600000, 0x0000000307800000| Untracked -| 56|0x0000000307800000, 0x0000000307a00000, 0x0000000307a00000|100%| O| |TAMS 0x0000000307800000, 0x0000000307a00000| Untracked -| 57|0x0000000307a00000, 0x0000000307c00000, 0x0000000307c00000|100%| O| |TAMS 0x0000000307a00000, 0x0000000307c00000| Untracked -| 58|0x0000000307c00000, 0x0000000307e00000, 0x0000000307e00000|100%| O| |TAMS 0x0000000307c00000, 0x0000000307e00000| Untracked -| 59|0x0000000307e00000, 0x0000000308000000, 0x0000000308000000|100%| O| |TAMS 0x0000000307e00000, 0x0000000308000000| Untracked -| 60|0x0000000308000000, 0x00000003081e9c00, 0x0000000308200000| 95%| O| |TAMS 0x0000000308000000, 0x00000003081e9c00| Untracked -| 61|0x0000000308200000, 0x0000000308200000, 0x0000000308400000| 0%| F| |TAMS 0x0000000308200000, 0x0000000308200000| Untracked -| 62|0x0000000308400000, 0x0000000308400000, 0x0000000308600000| 0%| F| |TAMS 0x0000000308400000, 0x0000000308400000| Untracked -| 63|0x0000000308600000, 0x0000000308800000, 0x0000000308800000|100%| O| |TAMS 0x0000000308600000, 0x0000000308800000| Untracked -| 64|0x0000000308800000, 0x0000000308a00000, 0x0000000308a00000|100%| O| |TAMS 0x0000000308800000, 0x0000000308a00000| Untracked -| 65|0x0000000308a00000, 0x0000000308c00000, 0x0000000308c00000|100%| O| |TAMS 0x0000000308a00000, 0x0000000308c00000| Untracked -| 66|0x0000000308c00000, 0x0000000308e00000, 0x0000000308e00000|100%|HS| |TAMS 0x0000000308c00000, 0x0000000308c00000| Complete -| 67|0x0000000308e00000, 0x0000000309000000, 0x0000000309000000|100%|HC| |TAMS 0x0000000308e00000, 0x0000000308e00000| Complete -| 68|0x0000000309000000, 0x0000000309200000, 0x0000000309200000|100%|HC| |TAMS 0x0000000309000000, 0x0000000309000000| Complete -| 69|0x0000000309200000, 0x0000000309200000, 0x0000000309400000| 0%| F| |TAMS 0x0000000309200000, 0x0000000309200000| Untracked -| 70|0x0000000309400000, 0x0000000309400000, 0x0000000309600000| 0%| F| |TAMS 0x0000000309400000, 0x0000000309400000| Untracked -| 71|0x0000000309600000, 0x0000000309800000, 0x0000000309800000|100%| O| |TAMS 0x0000000309600000, 0x0000000309800000| Untracked -| 72|0x0000000309800000, 0x0000000309a00000, 0x0000000309a00000|100%| O| |TAMS 0x0000000309800000, 0x0000000309a00000| Untracked -| 73|0x0000000309a00000, 0x0000000309c00000, 0x0000000309c00000|100%| O| |TAMS 0x0000000309a00000, 0x0000000309c00000| Untracked -| 74|0x0000000309c00000, 0x0000000309e00000, 0x0000000309e00000|100%| O| |TAMS 0x0000000309c00000, 0x0000000309e00000| Untracked -| 75|0x0000000309e00000, 0x000000030a000000, 0x000000030a000000|100%| O| |TAMS 0x0000000309e00000, 0x000000030a000000| Untracked -| 76|0x000000030a000000, 0x000000030a200000, 0x000000030a200000|100%| O| |TAMS 0x000000030a000000, 0x000000030a200000| Untracked -| 77|0x000000030a200000, 0x000000030a400000, 0x000000030a400000|100%| O| |TAMS 0x000000030a200000, 0x000000030a400000| Untracked -| 78|0x000000030a400000, 0x000000030a600000, 0x000000030a600000|100%| O| |TAMS 0x000000030a400000, 0x000000030a600000| Untracked -| 79|0x000000030a600000, 0x000000030a600000, 0x000000030a800000| 0%| F| |TAMS 0x000000030a600000, 0x000000030a600000| Untracked -| 80|0x000000030a800000, 0x000000030a800000, 0x000000030aa00000| 0%| F| |TAMS 0x000000030a800000, 0x000000030a800000| Untracked -| 81|0x000000030aa00000, 0x000000030aa00000, 0x000000030ac00000| 0%| F| |TAMS 0x000000030aa00000, 0x000000030aa00000| Untracked -| 82|0x000000030ac00000, 0x000000030ac00000, 0x000000030ae00000| 0%| F| |TAMS 0x000000030ac00000, 0x000000030ac00000| Untracked -| 83|0x000000030ae00000, 0x000000030ae00000, 0x000000030b000000| 0%| F| |TAMS 0x000000030ae00000, 0x000000030ae00000| Untracked -| 84|0x000000030b000000, 0x000000030b000000, 0x000000030b200000| 0%| F| |TAMS 0x000000030b000000, 0x000000030b000000| Untracked -| 85|0x000000030b200000, 0x000000030b200000, 0x000000030b400000| 0%| F| |TAMS 0x000000030b200000, 0x000000030b200000| Untracked -| 86|0x000000030b400000, 0x000000030b400000, 0x000000030b600000| 0%| F| |TAMS 0x000000030b400000, 0x000000030b400000| Untracked -| 87|0x000000030b600000, 0x000000030b600000, 0x000000030b800000| 0%| F| |TAMS 0x000000030b600000, 0x000000030b600000| Untracked -| 88|0x000000030b800000, 0x000000030b800000, 0x000000030ba00000| 0%| F| |TAMS 0x000000030b800000, 0x000000030b800000| Untracked -| 89|0x000000030ba00000, 0x000000030ba00000, 0x000000030bc00000| 0%| F| |TAMS 0x000000030ba00000, 0x000000030ba00000| Untracked -| 90|0x000000030bc00000, 0x000000030bc00000, 0x000000030be00000| 0%| F| |TAMS 0x000000030bc00000, 0x000000030bc00000| Untracked -| 91|0x000000030be00000, 0x000000030be00000, 0x000000030c000000| 0%| F| |TAMS 0x000000030be00000, 0x000000030be00000| Untracked -| 92|0x000000030c000000, 0x000000030c000000, 0x000000030c200000| 0%| F| |TAMS 0x000000030c000000, 0x000000030c000000| Untracked -| 93|0x000000030c200000, 0x000000030c200000, 0x000000030c400000| 0%| F| |TAMS 0x000000030c200000, 0x000000030c200000| Untracked -| 94|0x000000030c400000, 0x000000030c400000, 0x000000030c600000| 0%| F| |TAMS 0x000000030c400000, 0x000000030c400000| Untracked -| 95|0x000000030c600000, 0x000000030c600000, 0x000000030c800000| 0%| F| |TAMS 0x000000030c600000, 0x000000030c600000| Untracked -| 96|0x000000030c800000, 0x000000030c800000, 0x000000030ca00000| 0%| F| |TAMS 0x000000030c800000, 0x000000030c800000| Untracked -| 97|0x000000030ca00000, 0x000000030ca00000, 0x000000030cc00000| 0%| F| |TAMS 0x000000030ca00000, 0x000000030ca00000| Untracked -| 98|0x000000030cc00000, 0x000000030cc00000, 0x000000030ce00000| 0%| F| |TAMS 0x000000030cc00000, 0x000000030cc00000| Untracked -| 99|0x000000030ce00000, 0x000000030ce00000, 0x000000030d000000| 0%| F| |TAMS 0x000000030ce00000, 0x000000030ce00000| Untracked -| 100|0x000000030d000000, 0x000000030d000000, 0x000000030d200000| 0%| F| |TAMS 0x000000030d000000, 0x000000030d000000| Untracked -| 101|0x000000030d200000, 0x000000030d200000, 0x000000030d400000| 0%| F| |TAMS 0x000000030d200000, 0x000000030d200000| Untracked -| 102|0x000000030d400000, 0x000000030d400000, 0x000000030d600000| 0%| F| |TAMS 0x000000030d400000, 0x000000030d400000| Untracked -| 103|0x000000030d600000, 0x000000030d600000, 0x000000030d800000| 0%| F| |TAMS 0x000000030d600000, 0x000000030d600000| Untracked -| 104|0x000000030d800000, 0x000000030d800000, 0x000000030da00000| 0%| F| |TAMS 0x000000030d800000, 0x000000030d800000| Untracked -| 105|0x000000030da00000, 0x000000030da00000, 0x000000030dc00000| 0%| F| |TAMS 0x000000030da00000, 0x000000030da00000| Untracked -| 106|0x000000030dc00000, 0x000000030dc00000, 0x000000030de00000| 0%| F| |TAMS 0x000000030dc00000, 0x000000030dc00000| Untracked -| 107|0x000000030de00000, 0x000000030de00000, 0x000000030e000000| 0%| F| |TAMS 0x000000030de00000, 0x000000030de00000| Untracked -| 108|0x000000030e000000, 0x000000030e000000, 0x000000030e200000| 0%| F| |TAMS 0x000000030e000000, 0x000000030e000000| Untracked -| 109|0x000000030e200000, 0x000000030e200000, 0x000000030e400000| 0%| F| |TAMS 0x000000030e200000, 0x000000030e200000| Untracked -| 110|0x000000030e400000, 0x000000030e400000, 0x000000030e600000| 0%| F| |TAMS 0x000000030e400000, 0x000000030e400000| Untracked -| 111|0x000000030e600000, 0x000000030e600000, 0x000000030e800000| 0%| F| |TAMS 0x000000030e600000, 0x000000030e600000| Untracked -| 112|0x000000030e800000, 0x000000030e800000, 0x000000030ea00000| 0%| F| |TAMS 0x000000030e800000, 0x000000030e800000| Untracked -| 113|0x000000030ea00000, 0x000000030ea00000, 0x000000030ec00000| 0%| F| |TAMS 0x000000030ea00000, 0x000000030ea00000| Untracked -| 114|0x000000030ec00000, 0x000000030ec00000, 0x000000030ee00000| 0%| F| |TAMS 0x000000030ec00000, 0x000000030ec00000| Untracked -| 115|0x000000030ee00000, 0x000000030ee00000, 0x000000030f000000| 0%| F| |TAMS 0x000000030ee00000, 0x000000030ee00000| Untracked -| 116|0x000000030f000000, 0x000000030f000000, 0x000000030f200000| 0%| F| |TAMS 0x000000030f000000, 0x000000030f000000| Untracked -| 117|0x000000030f200000, 0x000000030f200000, 0x000000030f400000| 0%| F| |TAMS 0x000000030f200000, 0x000000030f200000| Untracked -| 118|0x000000030f400000, 0x000000030f400000, 0x000000030f600000| 0%| F| |TAMS 0x000000030f400000, 0x000000030f400000| Untracked -| 119|0x000000030f600000, 0x000000030f600000, 0x000000030f800000| 0%| F| |TAMS 0x000000030f600000, 0x000000030f600000| Untracked -| 120|0x000000030f800000, 0x000000030f800000, 0x000000030fa00000| 0%| F| |TAMS 0x000000030f800000, 0x000000030f800000| Untracked -| 121|0x000000030fa00000, 0x000000030fa00000, 0x000000030fc00000| 0%| F| |TAMS 0x000000030fa00000, 0x000000030fa00000| Untracked -| 122|0x000000030fc00000, 0x000000030fc00000, 0x000000030fe00000| 0%| F| |TAMS 0x000000030fc00000, 0x000000030fc00000| Untracked -| 123|0x000000030fe00000, 0x000000030fe00000, 0x0000000310000000| 0%| F| |TAMS 0x000000030fe00000, 0x000000030fe00000| Untracked -| 124|0x0000000310000000, 0x0000000310000000, 0x0000000310200000| 0%| F| |TAMS 0x0000000310000000, 0x0000000310000000| Untracked -| 125|0x0000000310200000, 0x0000000310200000, 0x0000000310400000| 0%| F| |TAMS 0x0000000310200000, 0x0000000310200000| Untracked -| 126|0x0000000310400000, 0x0000000310400000, 0x0000000310600000| 0%| F| |TAMS 0x0000000310400000, 0x0000000310400000| Untracked -| 127|0x0000000310600000, 0x0000000310600000, 0x0000000310800000| 0%| F| |TAMS 0x0000000310600000, 0x0000000310600000| Untracked -| 128|0x0000000310800000, 0x0000000310800000, 0x0000000310a00000| 0%| F| |TAMS 0x0000000310800000, 0x0000000310800000| Untracked -| 129|0x0000000310a00000, 0x0000000310a00000, 0x0000000310c00000| 0%| F| |TAMS 0x0000000310a00000, 0x0000000310a00000| Untracked -| 130|0x0000000310c00000, 0x0000000310c00000, 0x0000000310e00000| 0%| F| |TAMS 0x0000000310c00000, 0x0000000310c00000| Untracked -| 131|0x0000000310e00000, 0x0000000310e00000, 0x0000000311000000| 0%| F| |TAMS 0x0000000310e00000, 0x0000000310e00000| Untracked -| 132|0x0000000311000000, 0x0000000311000000, 0x0000000311200000| 0%| F| |TAMS 0x0000000311000000, 0x0000000311000000| Untracked -| 133|0x0000000311200000, 0x0000000311200000, 0x0000000311400000| 0%| F| |TAMS 0x0000000311200000, 0x0000000311200000| Untracked -| 134|0x0000000311400000, 0x0000000311400000, 0x0000000311600000| 0%| F| |TAMS 0x0000000311400000, 0x0000000311400000| Untracked -| 135|0x0000000311600000, 0x0000000311600000, 0x0000000311800000| 0%| F| |TAMS 0x0000000311600000, 0x0000000311600000| Untracked -| 136|0x0000000311800000, 0x0000000311800000, 0x0000000311a00000| 0%| F| |TAMS 0x0000000311800000, 0x0000000311800000| Untracked -| 137|0x0000000311a00000, 0x0000000311a00000, 0x0000000311c00000| 0%| F| |TAMS 0x0000000311a00000, 0x0000000311a00000| Untracked -| 138|0x0000000311c00000, 0x0000000311c00000, 0x0000000311e00000| 0%| F| |TAMS 0x0000000311c00000, 0x0000000311c00000| Untracked -| 139|0x0000000311e00000, 0x0000000311e00000, 0x0000000312000000| 0%| F| |TAMS 0x0000000311e00000, 0x0000000311e00000| Untracked -| 140|0x0000000312000000, 0x0000000312000000, 0x0000000312200000| 0%| F| |TAMS 0x0000000312000000, 0x0000000312000000| Untracked -| 141|0x0000000312200000, 0x0000000312200000, 0x0000000312400000| 0%| F| |TAMS 0x0000000312200000, 0x0000000312200000| Untracked -| 142|0x0000000312400000, 0x0000000312400000, 0x0000000312600000| 0%| F| |TAMS 0x0000000312400000, 0x0000000312400000| Untracked -| 143|0x0000000312600000, 0x0000000312600000, 0x0000000312800000| 0%| F| |TAMS 0x0000000312600000, 0x0000000312600000| Untracked -| 144|0x0000000312800000, 0x0000000312800000, 0x0000000312a00000| 0%| F| |TAMS 0x0000000312800000, 0x0000000312800000| Untracked -| 145|0x0000000312a00000, 0x0000000312a00000, 0x0000000312c00000| 0%| F| |TAMS 0x0000000312a00000, 0x0000000312a00000| Untracked -| 146|0x0000000312c00000, 0x0000000312c00000, 0x0000000312e00000| 0%| F| |TAMS 0x0000000312c00000, 0x0000000312c00000| Untracked -| 147|0x0000000312e00000, 0x0000000312e00000, 0x0000000313000000| 0%| F| |TAMS 0x0000000312e00000, 0x0000000312e00000| Untracked -| 148|0x0000000313000000, 0x0000000313000000, 0x0000000313200000| 0%| F| |TAMS 0x0000000313000000, 0x0000000313000000| Untracked -| 149|0x0000000313200000, 0x0000000313200000, 0x0000000313400000| 0%| F| |TAMS 0x0000000313200000, 0x0000000313200000| Untracked -| 150|0x0000000313400000, 0x0000000313400000, 0x0000000313600000| 0%| F| |TAMS 0x0000000313400000, 0x0000000313400000| Untracked -| 151|0x0000000313600000, 0x0000000313600000, 0x0000000313800000| 0%| F| |TAMS 0x0000000313600000, 0x0000000313600000| Untracked -| 152|0x0000000313800000, 0x00000003139375d0, 0x0000000313a00000| 60%| S|CS|TAMS 0x0000000313800000, 0x0000000313800000| Complete -| 153|0x0000000313a00000, 0x0000000313c00000, 0x0000000313c00000|100%| S|CS|TAMS 0x0000000313a00000, 0x0000000313a00000| Complete -| 154|0x0000000313c00000, 0x0000000313e00000, 0x0000000313e00000|100%| S|CS|TAMS 0x0000000313c00000, 0x0000000313c00000| Complete -| 155|0x0000000313e00000, 0x0000000314000000, 0x0000000314000000|100%| S|CS|TAMS 0x0000000313e00000, 0x0000000313e00000| Complete -| 156|0x0000000314000000, 0x0000000314000000, 0x0000000314200000| 0%| F| |TAMS 0x0000000314000000, 0x0000000314000000| Untracked -| 157|0x0000000314200000, 0x0000000314400000, 0x0000000314400000|100%| E| |TAMS 0x0000000314200000, 0x0000000314200000| Complete -| 158|0x0000000314400000, 0x0000000314600000, 0x0000000314600000|100%| E|CS|TAMS 0x0000000314400000, 0x0000000314400000| Complete -| 159|0x0000000314600000, 0x0000000314800000, 0x0000000314800000|100%| E|CS|TAMS 0x0000000314600000, 0x0000000314600000| Complete -| 160|0x0000000314800000, 0x0000000314a00000, 0x0000000314a00000|100%| E|CS|TAMS 0x0000000314800000, 0x0000000314800000| Complete -| 161|0x0000000314a00000, 0x0000000314c00000, 0x0000000314c00000|100%| E|CS|TAMS 0x0000000314a00000, 0x0000000314a00000| Complete -| 335|0x000000032a600000, 0x000000032a800000, 0x000000032a800000|100%| E|CS|TAMS 0x000000032a600000, 0x000000032a600000| Complete -| 336|0x000000032a800000, 0x000000032aa00000, 0x000000032aa00000|100%| E|CS|TAMS 0x000000032a800000, 0x000000032a800000| Complete -| 440|0x0000000337800000, 0x0000000337a00000, 0x0000000337a00000|100%| E|CS|TAMS 0x0000000337800000, 0x0000000337800000| Complete -| 441|0x0000000337a00000, 0x0000000337c00000, 0x0000000337c00000|100%| E|CS|TAMS 0x0000000337a00000, 0x0000000337a00000| Complete -| 442|0x0000000337c00000, 0x0000000337e00000, 0x0000000337e00000|100%| E|CS|TAMS 0x0000000337c00000, 0x0000000337c00000| Complete -| 767|0x0000000360600000, 0x0000000360800000, 0x0000000360800000|100%| E|CS|TAMS 0x0000000360600000, 0x0000000360600000| Complete -|2046|0x0000000400400000, 0x0000000400574000, 0x0000000400600000| 72%|OA| |TAMS 0x0000000400574000, 0x0000000400574000| Untracked -|2047|0x0000000400600000, 0x0000000400674000, 0x0000000400800000| 22%|CA| |TAMS 0x0000000400674000, 0x0000000400674000| Untracked - -Card table byte_map: [0x000000011e410000,0x000000011ec10000] _byte_map_base: 0x000000011cc0c000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x00000001014de750, (CMBitMap*) 0x00000001014de790 - Prev Bits: [0x000000011f410000, 0x0000000123410000) - Next Bits: [0x0000000125e04000, 0x0000000129e04000) - -Polling page: 0x0000000100dc4000 - -Metaspace: - -Usage: - Non-class: 130.17 MB used. - Class: 17.79 MB used. - Both: 147.96 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 130.88 MB ( 68%) committed, 3 nodes. - Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 149.25 MB ( 12%) committed. - -Chunk freelists: - Non-Class: 12.93 MB - Class: 13.67 MB - Both: 26.61 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 164.38 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 14. -num_arena_births: 2116. -num_arena_deaths: 0. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2385. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 14. -num_chunks_taken_from_freelist: 7831. -num_chunk_merges: 11. -num_chunk_splits: 5823. -num_chunks_enlarged: 4407. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=8762Kb max_used=9288Kb free=110405Kb - bounds [0x0000000116fb0000, 0x00000001178d0000, 0x000000011e410000] -CodeHeap 'profiled nmethods': size=119152Kb used=19803Kb max_used=20506Kb free=99349Kb - bounds [0x000000010f410000, 0x0000000110820000, 0x000000011686c000] -CodeHeap 'non-nmethods': size=7440Kb used=3322Kb max_used=3496Kb free=4117Kb - bounds [0x000000011686c000, 0x0000000116bec000, 0x0000000116fb0000] - total_blobs=11680 nmethods=10618 adapters=973 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 21.992 Thread 0x0000000758d82400 nmethod 14784 0x00000001102ee910 code [0x00000001102eeac0, 0x00000001102eecf8] -Event: 21.992 Thread 0x0000000758d82400 14790 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) -Event: 21.992 Thread 0x0000000752b5de00 nmethod 14788 0x00000001102ee290 code [0x00000001102ee480, 0x00000001102ee738] -Event: 21.992 Thread 0x0000000752b5de00 14792 3 org.apache.logging.slf4j.Log4jLogger::info (27 bytes) -Event: 21.992 Thread 0x0000000758d82400 nmethod 14790 0x00000001102edc90 code [0x00000001102ede80, 0x00000001102ee0e8] -Event: 21.992 Thread 0x0000000758d82400 14791 3 org.apache.logging.log4j.message.SimpleMessage::formatTo (43 bytes) -Event: 21.992 Thread 0x0000000752b5de00 nmethod 14792 0x000000010fee0a10 code [0x000000010fee0c00, 0x000000010fee0eb8] -Event: 21.992 Thread 0x0000000752b5de00 14793 3 org.apache.comet.ConfigEntry::key (13 bytes) -Event: 21.992 Thread 0x0000000752b5de00 nmethod 14793 0x000000010fee0590 code [0x000000010fee0740, 0x000000010fee08c8] -Event: 21.992 Thread 0x0000000752b5de00 14794 3 java.util.GregorianCalendar::computeFields (85 bytes) -Event: 21.992 Thread 0x0000000758d82400 nmethod 14791 0x000000010fedfe90 code [0x000000010fee0080, 0x000000010fee03b8] -Event: 21.992 Thread 0x0000000752b5de00 nmethod 14794 0x000000010fedf510 code [0x000000010fedf740, 0x000000010fedfb98] -Event: 21.993 Thread 0x0000000752b5d800 nmethod 14787 0x000000010f4d1a90 code [0x000000010f4d1e80, 0x000000010f4d3108] -Event: 21.993 Thread 0x0000000758d82400 14795 3 scala.collection.convert.Wrappers$IteratorWrapper::$jacocoInit (52 bytes) -Event: 21.993 Thread 0x0000000752b5e400 nmethod 14789 0x0000000110476910 code [0x0000000110476fc0, 0x000000011047a3e8] -Event: 21.993 Thread 0x0000000758d82400 nmethod 14795 0x0000000110439f90 code [0x000000011043a200, 0x000000011043abb8] -Event: 21.993 Thread 0x0000000752b5de00 14796 3 org.apache.comet.CometExecIterator$$$Lambda$3896/0x000000f8021abc98::apply (12 bytes) -Event: 21.993 Thread 0x0000000752b5d800 14797 3 org.apache.comet.CometExecIterator$::$anonfun$serializeCometSQLConfs$1 (159 bytes) -Event: 21.993 Thread 0x0000000752b5de00 nmethod 14796 0x0000000110439a90 code [0x0000000110439c40, 0x0000000110439e88] -Event: 21.993 Thread 0x0000000758d82400 14798 3 com.google.protobuf.MapEntry::writeTo (26 bytes) - -GC Heap History (20 events): -Event: 7.549 GC heap before -{Heap before GC invocations=28 (full 0): - garbage-first heap total 348160K, used 284481K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 66 young (135168K), 2 survivors (4096K) - Metaspace used 140778K, committed 141760K, reserved 1179648K - class space used 16878K, committed 17344K, reserved 1048576K -} -Event: 7.551 GC heap after -{Heap after GC invocations=29 (full 0): - garbage-first heap total 348160K, used 152858K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 140778K, committed 141760K, reserved 1179648K - class space used 16878K, committed 17344K, reserved 1048576K -} -Event: 7.727 GC heap before -{Heap before GC invocations=29 (full 0): - garbage-first heap total 348160K, used 290074K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 68 young (139264K), 3 survivors (6144K) - Metaspace used 147454K, committed 148544K, reserved 1179648K - class space used 17571K, committed 18048K, reserved 1048576K -} -Event: 7.728 GC heap after -{Heap after GC invocations=30 (full 0): - garbage-first heap total 348160K, used 154595K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 147454K, committed 148544K, reserved 1179648K - class space used 17571K, committed 18048K, reserved 1048576K -} -Event: 8.064 GC heap before -{Heap before GC invocations=30 (full 0): - garbage-first heap total 348160K, used 289763K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 70 young (143360K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.066 GC heap after -{Heap after GC invocations=31 (full 0): - garbage-first heap total 348160K, used 154934K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.335 GC heap before -{Heap before GC invocations=31 (full 0): - garbage-first heap total 348160K, used 292150K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 71 young (145408K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.337 GC heap after -{Heap after GC invocations=32 (full 0): - garbage-first heap total 348160K, used 155446K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.625 GC heap before -{Heap before GC invocations=32 (full 0): - garbage-first heap total 348160K, used 296758K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 73 young (149504K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.626 GC heap after -{Heap after GC invocations=33 (full 0): - garbage-first heap total 348160K, used 155007K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.917 GC heap before -{Heap before GC invocations=33 (full 0): - garbage-first heap total 348160K, used 298367K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 74 young (151552K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 8.918 GC heap after -{Heap after GC invocations=34 (full 0): - garbage-first heap total 348160K, used 155261K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 9.204 GC heap before -{Heap before GC invocations=34 (full 0): - garbage-first heap total 348160K, used 300669K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 75 young (153600K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 9.206 GC heap after -{Heap after GC invocations=35 (full 0): - garbage-first heap total 348160K, used 155102K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148866K, committed 150016K, reserved 1245184K - class space used 17780K, committed 18304K, reserved 1048576K -} -Event: 21.867 GC heap before -{Heap before GC invocations=35 (full 0): - garbage-first heap total 348160K, used 298462K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 73 young (149504K), 4 survivors (8192K) - Metaspace used 149284K, committed 150400K, reserved 1245184K - class space used 17854K, committed 18368K, reserved 1048576K -} -Event: 21.869 GC heap after -{Heap after GC invocations=36 (full 0): - garbage-first heap total 348160K, used 155067K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 149284K, committed 150400K, reserved 1245184K - class space used 17854K, committed 18368K, reserved 1048576K -} -Event: 21.905 GC heap before -{Heap before GC invocations=36 (full 0): - garbage-first heap total 348160K, used 161211K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 6 young (12288K), 4 survivors (8192K) - Metaspace used 149340K, committed 150464K, reserved 1245184K - class space used 17854K, committed 18368K, reserved 1048576K -} -Event: 21.907 GC heap after -{Heap after GC invocations=37 (full 0): - garbage-first heap total 348160K, used 155089K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 149340K, committed 150464K, reserved 1245184K - class space used 17854K, committed 18368K, reserved 1048576K -} -Event: 21.957 GC heap before -{Heap before GC invocations=37 (full 0): - garbage-first heap total 348160K, used 171473K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 12 young (24576K), 4 survivors (8192K) - Metaspace used 150237K, committed 151424K, reserved 1245184K - class space used 17984K, committed 18496K, reserved 1048576K -} -Event: 21.959 GC heap after -{Heap after GC invocations=38 (full 0): - garbage-first heap total 348160K, used 155166K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 150237K, committed 151424K, reserved 1245184K - class space used 17984K, committed 18496K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.514 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li -Event: 3.514 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.514 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.575 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14714718128898053499.dylib -Event: 6.098 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 6.098 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 6.098 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 6.098 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 6.098 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 6.098 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 6.098 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 6.098 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 6.098 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 6.098 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 6.098 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 6.098 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 6.098 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 6.098 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 6.219 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-12903721131319273091.dylib -Event: 6.983 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-6a22215f-8250-484c-aa43-855ee20024fe-libsnappyjava.dylib - -Deoptimization events (20 events): -Event: 21.961 Thread 0x000000075839de00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x0000000117807de4 relative=0x00000000000006a4 -Event: 21.961 Thread 0x000000075839de00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x0000000117807de4 method=scala.runtime.BoxesRunTime.equalsNumNum(Ljava/lang/Number;Ljava/lang/Number;)Z @ 84 c2 -Event: 21.961 Thread 0x000000075839de00 DEOPT PACKING pc=0x0000000117807de4 sp=0x000000017cb756e0 -Event: 21.961 Thread 0x000000075839de00 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000017cb754f0 mode 2 -Event: 21.961 Thread 0x000000075839de00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x0000000117809410 relative=0x0000000000000650 -Event: 21.961 Thread 0x000000075839de00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x0000000117809410 method=scala.runtime.BoxesRunTime.equalsNumNum(Ljava/lang/Number;Ljava/lang/Number;)Z @ 84 c2 -Event: 21.961 Thread 0x000000075839de00 DEOPT PACKING pc=0x0000000117809410 sp=0x000000017cb755c0 -Event: 21.961 Thread 0x000000075839de00 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000017cb75440 mode 2 -Event: 21.969 Thread 0x0000000755abd800 Uncommon trap: trap_request=0xfffffff4 fr.pc=0x00000001175d999c relative=0x000000000000045c -Event: 21.969 Thread 0x0000000755abd800 Uncommon trap: reason=null_check action=make_not_entrant pc=0x00000001175d999c method=java.io.ObjectInputStream.readArray(Z)Ljava/lang/Object; @ 392 c2 -Event: 21.969 Thread 0x0000000755abd800 DEOPT PACKING pc=0x00000001175d999c sp=0x0000000401010070 -Event: 21.969 Thread 0x0000000755abd800 DEOPT UNPACKING pc=0x00000001168b391c sp=0x000000040100ffd0 mode 2 -Event: 21.982 Thread 0x0000000755abe400 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011785ab90 relative=0x0000000000000810 -Event: 21.982 Thread 0x0000000755abe400 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011785ab90 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 -Event: 21.982 Thread 0x0000000755abe400 DEOPT PACKING pc=0x000000011785ab90 sp=0x0000000401828830 -Event: 21.982 Thread 0x0000000755abe400 DEOPT UNPACKING pc=0x00000001168b391c sp=0x00000004018286a0 mode 2 -Event: 21.987 Thread 0x0000000755abe400 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011785ab90 relative=0x0000000000000810 -Event: 21.987 Thread 0x0000000755abe400 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011785ab90 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 -Event: 21.987 Thread 0x0000000755abe400 DEOPT PACKING pc=0x000000011785ab90 sp=0x0000000401828850 -Event: 21.987 Thread 0x0000000755abe400 DEOPT UNPACKING pc=0x00000001168b391c sp=0x00000004018286a0 mode 2 - -Classes loaded (20 events): -Event: 7.699 Loading class java/lang/Class$EnclosingMethodInfo -Event: 7.699 Loading class java/lang/Class$EnclosingMethodInfo done -Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper -Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$Unmapper -Event: 16.706 Loading class jdk/internal/access/foreign/UnmapperProxy -Event: 16.706 Loading class jdk/internal/access/foreign/UnmapperProxy done -Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$Unmapper done -Event: 16.706 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done -Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask -Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask -Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done -Event: 19.932 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done -Event: 19.932 Loading class java/util/function/LongBinaryOperator -Event: 19.932 Loading class java/util/function/LongBinaryOperator done -Event: 21.793 Loading class sun/nio/ch/ChannelInputStream -Event: 21.794 Loading class sun/nio/ch/ChannelInputStream done -Event: 21.922 Loading class java/math/BigDecimal$StringBuilderHelper -Event: 21.922 Loading class java/math/BigDecimal$StringBuilderHelper done -Event: 21.988 Loading class java/nio/channels/Channels$ReadableByteChannelImpl -Event: 21.988 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 21.984 Thread 0x0000000755abde00 Exception (0x0000000314958d78) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.984 Thread 0x0000000755abe400 Exception (0x00000003149ca8c8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.984 Thread 0x0000000755abd800 Exception (0x0000000314be81c8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.984 Thread 0x0000000755abd200 Exception (0x0000000314b5d518) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.984 Thread 0x0000000755abea00 Exception (0x00000003148ebf68) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c18c8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b72188) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314711de8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314bfc940) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abde00 Exception (0x000000031496cf48) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c58b8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b75f60) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abde00 Exception (0x0000000314970d20) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abea00 Exception (0x00000003147c9198) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314715bc0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314400d70) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd200 Exception (0x0000000314b797f0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abde00 Exception (0x00000003149745b0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abe400 Exception (0x0000000314719450) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.986 Thread 0x0000000755abd800 Exception (0x0000000314404600) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] - -VM Operations (20 events): -Event: 17.246 Executing VM operation: Cleanup -Event: 17.246 Executing VM operation: Cleanup done -Event: 20.261 Executing VM operation: Cleanup -Event: 20.261 Executing VM operation: Cleanup done -Event: 21.849 Executing VM operation: HandshakeAllThreads -Event: 21.849 Executing VM operation: HandshakeAllThreads done -Event: 21.849 Executing VM operation: Cleanup -Event: 21.849 Executing VM operation: Cleanup done -Event: 21.864 Executing VM operation: HandshakeAllThreads -Event: 21.864 Executing VM operation: HandshakeAllThreads done -Event: 21.867 Executing VM operation: G1TryInitiateConcMark -Event: 21.869 Executing VM operation: G1TryInitiateConcMark done -Event: 21.895 Executing VM operation: HandshakeAllThreads -Event: 21.895 Executing VM operation: HandshakeAllThreads done -Event: 21.905 Executing VM operation: G1TryInitiateConcMark -Event: 21.907 Executing VM operation: G1TryInitiateConcMark done -Event: 21.948 Executing VM operation: ICBufferFull -Event: 21.948 Executing VM operation: ICBufferFull done -Event: 21.957 Executing VM operation: G1TryInitiateConcMark -Event: 21.959 Executing VM operation: G1TryInitiateConcMark done - -Memory protections (20 events): -Event: 7.559 Protecting memory [0x0000000403488000,0x0000000403494000] with protection modes 0 -Event: 7.559 Protecting memory [0x0000000403894000,0x00000004038a0000] with protection modes 0 -Event: 9.475 Protecting memory [0x0000000176430000,0x000000017643c000] with protection modes 3 -Event: 9.475 Protecting memory [0x0000000173734000,0x0000000173740000] with protection modes 3 -Event: 9.475 Protecting memory [0x0000000172af8000,0x0000000172b04000] with protection modes 3 -Event: 9.478 Protecting memory [0x00000001728ec000,0x00000001728f8000] with protection modes 3 -Event: 9.478 Protecting memory [0x00000001726e0000,0x00000001726ec000] with protection modes 3 -Event: 14.483 Protecting memory [0x00000001724d4000,0x00000001724e0000] with protection modes 3 -Event: 19.931 Protecting memory [0x00000001722c8000,0x00000001722d4000] with protection modes 3 -Event: 21.796 Protecting memory [0x00000001722c8000,0x00000001722d4000] with protection modes 0 -Event: 21.849 Protecting memory [0x00000001724d4000,0x00000001724e0000] with protection modes 0 -Event: 21.849 Protecting memory [0x00000001726e0000,0x00000001726ec000] with protection modes 0 -Event: 21.849 Protecting memory [0x00000001728ec000,0x00000001728f8000] with protection modes 0 -Event: 21.896 Protecting memory [0x0000000172af8000,0x0000000172b04000] with protection modes 0 -Event: 21.911 Protecting memory [0x00000004044d0000,0x00000004044dc000] with protection modes 0 -Event: 21.960 Protecting memory [0x00000004048dc000,0x00000004048e8000] with protection modes 0 -Event: 21.961 Protecting memory [0x0000000404d00000,0x0000000404d0c000] with protection modes 0 -Event: 21.961 Protecting memory [0x000000040510c000,0x0000000405118000] with protection modes 0 -Event: 21.961 Protecting memory [0x0000000405518000,0x0000000405524000] with protection modes 0 -Event: 21.961 Protecting memory [0x0000000405924000,0x0000000405930000] with protection modes 0 - -Nmethod flushes (20 events): -Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x000000011079af90 -Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x000000011079f690 -Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a5990 -Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a5e10 -Event: 21.903 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a6490 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107a9e90 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107aa490 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107aad90 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ac210 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ae910 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107afd10 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b1590 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b2c10 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b3510 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b3910 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107b5e90 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107ba310 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d7690 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d8510 -Event: 21.904 Thread 0x0000000758d82a00 flushing nmethod 0x00000001107d8990 - -Events (20 events): -Event: 7.559 Thread 0x0000000752907600 Thread added: 0x0000000752974000 -Event: 7.559 Thread 0x0000000752974000 Thread added: 0x0000000752974600 -Event: 9.475 Thread 0x0000000756ff3000 Thread exited: 0x0000000756ff3000 -Event: 9.475 Thread 0x0000000755abf000 Thread exited: 0x0000000755abf000 -Event: 9.475 Thread 0x0000000755acb000 Thread exited: 0x0000000755acb000 -Event: 9.478 Thread 0x0000000754d86400 Thread exited: 0x0000000754d86400 -Event: 9.478 Thread 0x0000000754d4de00 Thread exited: 0x0000000754d4de00 -Event: 14.483 Thread 0x000000075598ea00 Thread exited: 0x000000075598ea00 -Event: 19.931 Thread 0x0000000758d83600 Thread exited: 0x0000000758d83600 -Event: 21.796 Thread 0x0000000758d82400 Thread added: 0x0000000752b5d200 -Event: 21.849 Thread 0x0000000752b5d200 Thread added: 0x0000000752b5d800 -Event: 21.849 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5de00 -Event: 21.849 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5e400 -Event: 21.896 Thread 0x0000000752b5d800 Thread added: 0x0000000752b5ea00 -Event: 21.911 Thread 0x000000075839de00 Thread added: 0x0000000758e9f600 -Event: 21.960 Thread 0x0000000758375800 Thread added: 0x0000000755abf000 -Event: 21.961 Thread 0x0000000755abf000 Thread added: 0x0000000755abcc00 -Event: 21.961 Thread 0x0000000755abf000 Thread added: 0x0000000755acb000 -Event: 21.961 Thread 0x0000000755acb000 Thread added: 0x0000000756ff3000 -Event: 21.961 Thread 0x0000000756ff3000 Thread added: 0x00000007535b4000 - - -Dynamic libraries: -0x0000000100ddc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x00000001068cc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x0000000100e04000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x0000000100e34000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x0000000100e84000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x0000000100f20000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x0000000100f48000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x0000000100f68000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x0000000100f8c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x0000000100f0c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x0000000100fa8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x0000000100fbc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x0000000106444000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000145dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14714718128898053499.dylib -0x0000000106514000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-12903721131319273091.dylib -0x0000000106540000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-6a22215f-8250-484c-aa43-855ee20024fe-libsnappyjava.dylib - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:38 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 10.87 7.26 5.05 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(1982112k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. diff --git a/spark/hs_err_pid58919.log b/spark/hs_err_pid58919.log deleted file mode 100644 index b6306fa045..0000000000 --- a/spark/hs_err_pid58919.log +++ /dev/null @@ -1,1602 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x000000010af31f64, pid=58919, tid=86283 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 08:57:06 2026 MDT elapsed time: 20.876276 seconds (0d 0h 0m 20s) - ---------------- T H R E A D --------------- - -Current thread (0x00000008d513e400): JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=86283, stack(0x000000017c130000,0x000000017c533000)] - -Stack: [0x000000017c130000,0x000000017c533000], sp=0x000000017c51a120, free space=4008k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -C [libcomet-16636361560025204640.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 -C [libcomet-16636361560025204640.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 -C [libcomet-16636361560025204640.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c -C [libcomet-16636361560025204640.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 -C [libcomet-16636361560025204640.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 -C [libcomet-16636361560025204640.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 -C [libcomet-16636361560025204640.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 -C [libcomet-16636361560025204640.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-16636361560025204640.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-16636361560025204640.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-16636361560025204640.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-16636361560025204640.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-16636361560025204640.dylib+0x106628] __rust_try+0x20 -C [libcomet-16636361560025204640.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-16636361560025204640.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-16636361560025204640.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-16636361560025204640.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3952+0x00000005021bf940.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3946+0x00000005021beb30.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2517+0x0000000501df17d0.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3952+0x00000005021bf940.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3946+0x00000005021beb30.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2517+0x0000000501df17d0.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000234b24 - -Registers: - x0=0x00000005021ae260 x1=0x0000000000000010 x2=0x00000008c6440fe0 x3=0x000000017c51f298 - x4=0x0000000000000238 x5=0x0000000000000018 x6=0x00000008c4852220 x7=0xfffff0003ffff800 - x8=0x00000000021ae260 x9=0x0000000000000000 x10=0x0000000500000000 x11=0x00000008c90485c0 -x12=0x00000008cc1300a0 x13=0x0000000000000001 x14=0x000000010539cb98 x15=0x00000008c9048000 -x16=0x000000018c8009d0 x17=0x00000008c8fcc500 x18=0x0000000000000000 x19=0x00000008d513e400 -x20=0x0000000000000001 x21=0x00000008d2a00c80 x22=0x0000000000234b10 x23=0x000000010b691088 -x24=0x0000000000000000 x25=0x000000010b695f38 x26=0x0000000138fc7948 x27=0x00000003034a7f58 -x28=0x00000008d513e400 fp=0x000000017c51a160 lr=0x000000010af31f38 sp=0x000000017c51a120 -pc=0x000000010af31f64 cpsr=0x0000000040001000 - -Register to memory mapping: - - x0=0x00000005021ae260 is a pointer to class: -org.apache.comet.CometHandleBatchIterator {0x00000005021ae260} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x0000000139041ae0) - - method ordering: Array(0x0000000500913f08) - - local interfaces: Array(0x00000005005d4fb8) - - trans. interfaces: Array(0x00000005005d4fb8) - - constants: constant pool [62]/operands[4] {0x0000000139041850} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000139041d30 - - class loader data: loader data: 0x00000008c909d0e0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x0000000303323e00} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x00000005005d4f48) - - nest members: Array(0x00000005005d4f48) - - permitted subclasses: Array(0x00000005005d4f48) - - java mirror: a 'java/lang/Class'{0x000000030b4c7778} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x00000005021ae430) - - itable length 2 (start addr: 0x00000005021ae460) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x1=0x0000000000000010 is an unknown value - x2=0x00000008c6440fe0 points into unknown readable memory: 0x000000030c2a0ba0 | a0 0b 2a 0c 03 00 00 00 - x3=0x000000017c51f298 is pointing into the stack for thread: 0x00000008d513e400 - x4=0x0000000000000238 is an unknown value - x5=0x0000000000000018 is an unknown value - x6=0x00000008c4852220 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=35316320 is a compressed pointer to class: 0x00000005021ae260 -org.apache.comet.CometHandleBatchIterator {0x00000005021ae260} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x0000000139041ae0) - - method ordering: Array(0x0000000500913f08) - - local interfaces: Array(0x00000005005d4fb8) - - trans. interfaces: Array(0x00000005005d4fb8) - - constants: constant pool [62]/operands[4] {0x0000000139041850} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000139041d30 - - class loader data: loader data: 0x00000008c909d0e0 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x0000000303323e00} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x00000005005d4f48) - - nest members: Array(0x00000005005d4f48) - - permitted subclasses: Array(0x00000005005d4f48) - - java mirror: a 'java/lang/Class'{0x000000030b4c7778} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x00000005021ae430) - - itable length 2 (start addr: 0x00000005021ae460) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x9=0x0 is NULL -x10=0x0000000500000000 is pointing into metadata -x11=0x00000008c90485c0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -x12=0x00000008cc1300a0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x13=0x0000000000000001 is an unknown value -x14=0x000000010539cb98 points into unknown readable memory: 0x30eb1755804499e0 | e0 99 44 80 55 17 eb 30 -x15=0x00000008c9048000 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 -x17=0x00000008c8fcc500 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -x18=0x0 is NULL -x19=0x00000008d513e400 is a thread -x20=0x0000000000000001 is an unknown value -x21=0x00000008d2a00c80 is a global jni handle -x22=0x0000000000234b10 is an unknown value -x23=0x000000010b691088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 -x24=0x0 is NULL -x25=0x000000010b695f38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 -x26=0x0000000138fc7948 is pointing into metadata -x27=0x00000003034a7f58 is an oop: [Z -{0x00000003034a7f58} - klass: {type array bool} - - length: 9 -x28=0x00000008d513e400 is a thread - - -Top of Stack: (sp=0x000000017c51a120) -0x000000017c51a120: 0000000138fc7948 000000030334d9d0 -0x000000017c51a130: 000000017c531818 000000030334d3b0 -0x000000017c51a140: 0000000000000000 000000010b6dff90 -0x000000017c51a150: 000000017c531738 000000011aa1b72c -0x000000017c51a160: 000000017c51a5f0 000000040717ce4c -0x000000017c51a170: 0000000000000000 000000040a09e880 -0x000000017c51a180: 000000017c51a190 00000004071b6d54 -0x000000017c51a190: 00000008c6440fe0 00000008d2a00c80 -0x000000017c51a1a0: 000000010af31e08 00000008d719ccd0 -0x000000017c51a1b0: 000000010b69aad0 000000010b69aad0 -0x000000017c51a1c0: 00000008d513e6b0 00000008d513e6b0 -0x000000017c51a1d0: 000000040a0aae90 000000000717ee14 -0x000000017c51a1e0: 0000000000000000 000000017c51a270 -0x000000017c51a1f0: 00000008d513e6b0 000000040a0aae90 -0x000000017c51a200: 00000000d513e400 000000017c51a860 -0x000000017c51a210: 000000017c51aa00 8000000000000020 -0x000000017c51a220: 000000040a0aae90 00000008d719ccd0 -0x000000017c51a230: 000000040a0aae90 8000000000000020 -0x000000017c51a240: 000000040a0aae90 00000008d513e6b0 -0x000000017c51a250: 00000008d513e6b0 0000000000000002 -0x000000017c51a260: 00000008c8fcc500 0000000000000000 -0x000000017c51a270: 00000008d513e6b0 0000000000000002 -0x000000017c51a280: 00000008c8fcc500 8000000000000020 -0x000000017c51a290: 000000040a0aae90 00000008c8fcc5d0 -0x000000017c51a2a0: 000000017c51a2e0 00000004071dfc28 -0x000000017c51a2b0: 0000000000000000 0000000000000000 -0x000000017c51a2c0: 00000008d513e6b0 000000040a09e888 -0x000000017c51a2d0: 0000000000000000 00000008d513e6b0 -0x000000017c51a2e0: 00000008d513e6b0 0000000000000002 -0x000000017c51a2f0: 0000000000000008 000000040a09e888 -0x000000017c51a300: 000000017c51a340 00000004071b7d8c -0x000000017c51a310: 000000017c51a3f8 00000008d513e6b0 - -Instructions: (pc=0x000000010af31f64) -0x000000010af31e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 -0x000000010af31e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 -0x000000010af31e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 -0x000000010af31e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 -0x000000010af31ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 -0x000000010af31eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 -0x000000010af31ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 -0x000000010af31ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 -0x000000010af31ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 -0x000000010af31ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a -0x000000010af31f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 -0x000000010af31f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa -0x000000010af31f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 -0x000000010af31f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 -0x000000010af31f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 -0x000000010af31f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b -0x000000010af31f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 -0x000000010af31f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 -0x000000010af31f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 -0x000000010af31f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb -0x000000010af31fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 -0x000000010af31fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 -0x000000010af31fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 -0x000000010af31fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 -0x000000010af31fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 -0x000000010af31ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 -0x000000010af32004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b -0x000000010af32014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 -0x000000010af32024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 -0x000000010af32034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 -0x000000010af32044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 -0x000000010af32054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x0000000138fc7948 is pointing into metadata -stack at sp + 1 slots: 0x000000030334d9d0 is an oop: [Z -{0x000000030334d9d0} - klass: {type array bool} - - length: 264 -stack at sp + 2 slots: 0x000000017c531818 is pointing into the stack for thread: 0x00000008d513e400 -stack at sp + 3 slots: 0x000000030334d3b0 is an oop: [Z -{0x000000030334d3b0} - klass: {type array bool} - - length: 187 -stack at sp + 4 slots: 0x0 is NULL -stack at sp + 5 slots: 0x000000010b6dff90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x000000010aa74000 -stack at sp + 6 slots: 0x000000017c531738 is pointing into the stack for thread: 0x00000008d513e400 -stack at sp + 7 slots: 0x000000011aa1b72c is at code_begin+44 in an Interpreter codelet -result handlers for native calls [0x000000011aa1b700, 0x000000011aa1b780] 128 bytes - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x00000008c60b5f20, length=72, elements={ -0x000000010567a000, 0x000000010568e6d0, 0x00000008cc040000, 0x00000008cc040c00, -0x00000008cc041200, 0x00000008cc041800, 0x00000008cc041e00, 0x00000008cc042400, -0x00000008cc042a00, 0x00000008cc043000, 0x00000008cc17aa00, 0x00000008d27f0600, -0x00000008d27f2a00, 0x00000008d4c41800, 0x00000008d4c41200, 0x00000008d4c40c00, -0x00000008d4c42400, 0x00000008d47daa00, 0x00000008d4cc4c00, 0x00000008d4cc5200, -0x00000008d4cc5800, 0x00000008d4cc5e00, 0x00000008d4cc6400, 0x00000008d4cc6a00, -0x00000008d4cc7000, 0x00000008d4cc7600, 0x00000008d4ccde00, 0x00000008d4d00000, -0x00000008d4d00600, 0x00000008d4d01800, 0x00000008d4d01e00, 0x00000008d4d02a00, -0x00000008d4d03000, 0x00000008d4d03600, 0x00000008d4d14600, 0x00000008d4d14c00, -0x00000008d4d15e00, 0x00000008d4d16400, 0x00000008d4d17000, 0x00000008d4d28000, -0x00000008d4d28600, 0x00000008d4d28c00, 0x00000008d4d29200, 0x00000008d4d69800, -0x00000008d4d6b000, 0x00000008d4d94000, 0x00000008d4d96a00, 0x00000008d4b7c000, -0x00000008d513d800, 0x00000008d513de00, 0x00000008d513e400, 0x00000008d513ea00, -0x00000008d513f000, 0x00000008d5147600, 0x00000008ce8e2a00, 0x00000008d68d5800, -0x00000008cfaf4c00, 0x00000008d4bfea00, 0x00000008d4bfde00, 0x00000008cc179200, -0x00000008d5144000, 0x00000008d708ea00, 0x00000008d708f000, 0x00000008d708f600, -0x00000008d70e4000, 0x00000008d70e4600, 0x00000008d4bfc600, 0x00000008cc043600, -0x00000008cc179800, 0x00000008cc178000, 0x00000008d11a4c00, 0x00000008d70e7000 -} - -Java Threads: ( => current thread ) - 0x000000010567a000 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4611, stack(0x000000016afb8000,0x000000016b3bb000)] - 0x000000010568e6d0 JavaThread "Reference Handler" daemon [_thread_blocked, id=31747, stack(0x000000016c00c000,0x000000016c40f000)] - 0x00000008cc040000 JavaThread "Finalizer" daemon [_thread_blocked, id=23555, stack(0x000000016c418000,0x000000016c81b000)] - 0x00000008cc040c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30467, stack(0x000000016c93c000,0x000000016cd3f000)] - 0x00000008cc041200 JavaThread "Service Thread" daemon [_thread_blocked, id=26115, stack(0x000000016cd48000,0x000000016d14b000)] - 0x00000008cc041800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=29955, stack(0x000000016d154000,0x000000016d557000)] - 0x00000008cc041e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26627, stack(0x000000016d560000,0x000000016d763000)] - 0x00000008cc042400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27139, stack(0x000000016d76c000,0x000000016d96f000)] - 0x00000008cc042a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27395, stack(0x000000016d978000,0x000000016dd7b000)] - 0x00000008cc043000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27907, stack(0x000000016dd84000,0x000000016e187000)] - 0x00000008cc17aa00 JavaThread "Notification Thread" daemon [_thread_blocked, id=33539, stack(0x000000016edd8000,0x000000016f1db000)] - 0x00000008d27f0600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45059, stack(0x0000000171ce0000,0x00000001720e3000)] - 0x00000008d27f2a00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=63235, stack(0x0000000172b28000,0x0000000172f2b000)] - 0x00000008d4c41800 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=62979, stack(0x0000000172f34000,0x0000000173337000)] - 0x00000008d4c41200 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=62467, stack(0x0000000173340000,0x0000000173743000)] - 0x00000008d4c40c00 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=62211, stack(0x000000017374c000,0x0000000173b4f000)] - 0x00000008d4c42400 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=61699, stack(0x0000000173b58000,0x0000000173f5b000)] - 0x00000008d47daa00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=47619, stack(0x0000000173f64000,0x0000000174367000)] - 0x00000008d4cc4c00 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=48131, stack(0x0000000174370000,0x0000000174773000)] - 0x00000008d4cc5200 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=48387, stack(0x000000017477c000,0x0000000174b7f000)] - 0x00000008d4cc5800 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=48643, stack(0x0000000174b88000,0x0000000174f8b000)] - 0x00000008d4cc5e00 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=60675, stack(0x0000000174f94000,0x0000000175397000)] - 0x00000008d4cc6400 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=49155, stack(0x00000001753a0000,0x00000001757a3000)] - 0x00000008d4cc6a00 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=59907, stack(0x00000001757ac000,0x0000000175baf000)] - 0x00000008d4cc7000 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49667, stack(0x0000000175bb8000,0x0000000175fbb000)] - 0x00000008d4cc7600 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=49923, stack(0x0000000175fc4000,0x00000001763c7000)] - 0x00000008d4ccde00 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50179, stack(0x00000001763d0000,0x00000001767d3000)] - 0x00000008d4d00000 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58883, stack(0x00000001767dc000,0x0000000176bdf000)] - 0x00000008d4d00600 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=58371, stack(0x0000000176be8000,0x0000000176feb000)] - 0x00000008d4d01800 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58115, stack(0x0000000176ff4000,0x00000001773f7000)] - 0x00000008d4d01e00 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=57603, stack(0x0000000177400000,0x0000000177803000)] - 0x00000008d4d02a00 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=50947, stack(0x000000017780c000,0x0000000177c0f000)] - 0x00000008d4d03000 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57091, stack(0x0000000177c18000,0x000000017801b000)] - 0x00000008d4d03600 JavaThread "task-abort-timer" daemon [_thread_blocked, id=51203, stack(0x0000000178024000,0x0000000178427000)] - 0x00000008d4d14600 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=51459, stack(0x0000000178430000,0x0000000178833000)] - 0x00000008d4d14c00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=51715, stack(0x000000017883c000,0x0000000178c3f000)] - 0x00000008d4d15e00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=56067, stack(0x0000000178c48000,0x000000017904b000)] - 0x00000008d4d16400 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=55555, stack(0x0000000179054000,0x0000000179457000)] - 0x00000008d4d17000 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=55043, stack(0x0000000179460000,0x0000000179863000)] - 0x00000008d4d28000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54787, stack(0x000000017986c000,0x0000000179c6f000)] - 0x00000008d4d28600 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=52739, stack(0x0000000179c78000,0x000000017a07b000)] - 0x00000008d4d28c00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=53251, stack(0x000000017a084000,0x000000017a487000)] - 0x00000008d4d29200 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=53507, stack(0x000000017a490000,0x000000017a893000)] - 0x00000008d4d69800 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=54019, stack(0x000000017a89c000,0x000000017ac9f000)] - 0x00000008d4d6b000 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=65539, stack(0x000000017aca8000,0x000000017b0ab000)] - 0x00000008d4d94000 JavaThread "Timer-0" [_thread_blocked, id=66051, stack(0x000000017b0b4000,0x000000017b4b7000)] - 0x00000008d4d96a00 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=86787, stack(0x000000017b4c0000,0x000000017b8c3000)] - 0x00000008d4b7c000 JavaThread "process reaper" daemon [_thread_blocked, id=66563, stack(0x000000017bce4000,0x000000017bd1b000)] - 0x00000008d513d800 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=41779, stack(0x000000017b8cc000,0x000000017bccf000)] - 0x00000008d513de00 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=24851, stack(0x000000017bd24000,0x000000017c127000)] -=>0x00000008d513e400 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=86283, stack(0x000000017c130000,0x000000017c533000)] - 0x00000008d513ea00 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=85775, stack(0x000000017c53c000,0x000000017c93f000)] - 0x00000008d513f000 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=85507, stack(0x000000017c948000,0x000000017cd4b000)] - 0x00000008d5147600 JavaThread "process reaper" daemon [_thread_blocked, id=83971, stack(0x000000016c8b0000,0x000000016c8e7000)] - 0x00000008ce8e2a00 JavaThread "process reaper" daemon [_thread_blocked, id=83203, stack(0x000000016c8f0000,0x000000016c927000)] - 0x00000008d68d5800 JavaThread "process reaper" daemon [_thread_blocked, id=82691, stack(0x000000017d778000,0x000000017d7af000)] - 0x00000008cfaf4c00 JavaThread "process reaper" daemon [_thread_blocked, id=82179, stack(0x000000017d7b8000,0x000000017d7ef000)] - 0x00000008d4bfea00 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=42535, stack(0x000000017cd54000,0x000000017d157000)] - 0x00000008d4bfde00 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=67343, stack(0x000000017d160000,0x000000017d563000)] - 0x00000008cc179200 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=81971, stack(0x000000017d7f8000,0x000000017dbfb000)] - 0x00000008d5144000 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=68891, stack(0x000000017dc04000,0x000000017e007000)] - 0x00000008d708ea00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69155, stack(0x000000017e010000,0x000000017e413000)] - 0x00000008d708f000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=81731, stack(0x000000017e41c000,0x000000017e81f000)] - 0x00000008d708f600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=69379, stack(0x000000017e828000,0x000000017ec2b000)] - 0x00000008d70e4000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81155, stack(0x000000017ec34000,0x000000017f037000)] - 0x00000008d70e4600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=69635, stack(0x000000017f040000,0x000000017f443000)] - 0x00000008d4bfc600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=78095, stack(0x000000016e190000,0x000000016e393000)] - 0x00000008cc043600 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=79123, stack(0x000000016e39c000,0x000000016e59f000)] - 0x00000008cc179800 JavaThread "C2 CompilerThread2" daemon [_thread_in_native, id=78599, stack(0x000000016e5a8000,0x000000016e7ab000)] - 0x00000008cc178000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=43059, stack(0x000000016e7b4000,0x000000016e9b7000)] - 0x00000008d11a4c00 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=64051, stack(0x000000016e9c0000,0x000000016ebc3000)] - 0x00000008d70e7000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=45351, stack(0x00000004115d8000,0x00000004119db000)] - -Other Threads: - 0x000000010568da50 VMThread "VM Thread" [stack: 0x000000016be00000,0x000000016c003000] [id=20227] - 0x00000008cc07d200 WatcherThread [stack: 0x000000016f1e4000,0x000000016f3e7000] [id=34051] - 0x0000000105686440 GCTaskThread "GC Thread#0" [stack: 0x000000016b3c4000,0x000000016b5c7000] [id=14595] - 0x00000008cea71b00 GCTaskThread "GC Thread#1" [stack: 0x000000016f5fc000,0x000000016f7ff000] [id=34563] - 0x00000008cea71e00 GCTaskThread "GC Thread#2" [stack: 0x000000016f808000,0x000000016fa0b000] [id=41475] - 0x00000008cea72100 GCTaskThread "GC Thread#3" [stack: 0x000000016fa14000,0x000000016fc17000] [id=41219] - 0x00000008cea72400 GCTaskThread "GC Thread#4" [stack: 0x000000016fc20000,0x000000016fe23000] [id=35331] - 0x00000008cea72700 GCTaskThread "GC Thread#5" [stack: 0x000000016fe2c000,0x000000017002f000] [id=40707] - 0x00000008cea72a00 GCTaskThread "GC Thread#6" [stack: 0x0000000170038000,0x000000017023b000] [id=40195] - 0x00000008cea72d00 GCTaskThread "GC Thread#7" [stack: 0x0000000170244000,0x0000000170447000] [id=39939] - 0x00000008cea73000 GCTaskThread "GC Thread#8" [stack: 0x0000000170450000,0x0000000170653000] [id=36355] - 0x00000008cea73300 GCTaskThread "GC Thread#9" [stack: 0x000000017065c000,0x000000017085f000] [id=39427] - 0x00000008cea73600 GCTaskThread "GC Thread#10" [stack: 0x0000000170868000,0x0000000170a6b000] [id=36867] - 0x00000008cea73900 GCTaskThread "GC Thread#11" [stack: 0x0000000170a74000,0x0000000170c77000] [id=39171] - 0x00000008cea73c00 GCTaskThread "GC Thread#12" [stack: 0x0000000170c80000,0x0000000170e83000] [id=38659] - 0x00000008cff9c000 GCTaskThread "GC Thread#13" [stack: 0x0000000170e8c000,0x000000017108f000] [id=37379] - 0x00000008cff9c300 GCTaskThread "GC Thread#14" [stack: 0x0000000171098000,0x000000017129b000] [id=37635] - 0x00000008cff9c600 GCTaskThread "GC Thread#15" [stack: 0x00000001712a4000,0x00000001714a7000] [id=43523] - 0x00000008cff9c900 GCTaskThread "GC Thread#16" [stack: 0x00000001714b0000,0x00000001716b3000] [id=65027] - 0x00000008cff9cc00 GCTaskThread "GC Thread#17" [stack: 0x00000001716bc000,0x00000001718bf000] [id=64771] - 0x00000008cff9cf00 GCTaskThread "GC Thread#18" [stack: 0x00000001718c8000,0x0000000171acb000] [id=44291] - 0x00000008cff9d200 GCTaskThread "GC Thread#19" [stack: 0x0000000171ad4000,0x0000000171cd7000] [id=44547] - 0x0000000105686d40 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016b5d0000,0x000000016b7d3000] [id=14339] - 0x0000000105687640 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016b7dc000,0x000000016b9df000] [id=14083] - 0x00000008d47c8c00 ConcurrentGCThread "G1 Conc#1" [stack: 0x00000001722f8000,0x00000001724fb000] [id=45571] - 0x00000008d47c9200 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000172504000,0x0000000172707000] [id=63747] - 0x00000008d47c9500 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000172710000,0x0000000172913000] [id=46339] - 0x00000008d47c9800 ConcurrentGCThread "G1 Conc#4" [stack: 0x000000017291c000,0x0000000172b1f000] [id=63491] - 0x00000001056897e0 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016b9e8000,0x000000016bbeb000] [id=16643] - 0x000000010568a0e0 ConcurrentGCThread "G1 Service" [stack: 0x000000016bbf4000,0x000000016bdf7000] [id=21507] - -Threads with active compile tasks: -C2 CompilerThread2 20989 14625 4 java.io.ObjectInputStream$FieldValues:: (217 bytes) - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x0000000300800000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x0000000500000000-0x0000000500bc0000-0x0000000500bc0000), size 12320768, SharedBaseAddress: 0x0000000500000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x0000000501000000-0x0000000541000000, reserved size: 1073741824 -Narrow klass base: 0x0000000500000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 342016K, used 246166K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 51 young (104448K), 5 survivors (10240K) - Metaspace used 151361K, committed 152640K, reserved 1245184K - class space used 18196K, committed 18752K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000300800000, 0x0000000300a00000, 0x0000000300a00000|100%|HS| |TAMS 0x0000000300a00000, 0x0000000300800000| Complete -| 1|0x0000000300a00000, 0x0000000300c00000, 0x0000000300c00000|100%|HS| |TAMS 0x0000000300c00000, 0x0000000300a00000| Complete -| 2|0x0000000300c00000, 0x0000000300e00000, 0x0000000300e00000|100%|HC| |TAMS 0x0000000300e00000, 0x0000000300c00000| Complete -| 3|0x0000000300e00000, 0x0000000301000000, 0x0000000301000000|100%|HS| |TAMS 0x0000000301000000, 0x0000000300e00000| Complete -| 4|0x0000000301000000, 0x0000000301200000, 0x0000000301200000|100%|HC| |TAMS 0x0000000301200000, 0x0000000301000000| Complete -| 5|0x0000000301200000, 0x0000000301400000, 0x0000000301400000|100%|HC| |TAMS 0x0000000301400000, 0x0000000301200000| Complete -| 6|0x0000000301400000, 0x0000000301600000, 0x0000000301600000|100%|HC| |TAMS 0x0000000301600000, 0x0000000301400000| Complete -| 7|0x0000000301600000, 0x0000000301800000, 0x0000000301800000|100%|HC| |TAMS 0x0000000301800000, 0x0000000301600000| Complete -| 8|0x0000000301800000, 0x0000000301a00000, 0x0000000301a00000|100%|HC| |TAMS 0x0000000301a00000, 0x0000000301800000| Complete -| 9|0x0000000301a00000, 0x0000000301c00000, 0x0000000301c00000|100%|HC| |TAMS 0x0000000301c00000, 0x0000000301a00000| Complete -| 10|0x0000000301c00000, 0x0000000301e00000, 0x0000000301e00000|100%|HC| |TAMS 0x0000000301e00000, 0x0000000301c00000| Complete -| 11|0x0000000301e00000, 0x0000000302000000, 0x0000000302000000|100%|HS| |TAMS 0x0000000302000000, 0x0000000301e00000| Complete -| 12|0x0000000302000000, 0x0000000302200000, 0x0000000302200000|100%|HS| |TAMS 0x0000000302200000, 0x0000000302000000| Complete -| 13|0x0000000302200000, 0x0000000302400000, 0x0000000302400000|100%|HC| |TAMS 0x0000000302400000, 0x0000000302200000| Complete -| 14|0x0000000302400000, 0x0000000302600000, 0x0000000302600000|100%|HS| |TAMS 0x0000000302600000, 0x0000000302400000| Complete -| 15|0x0000000302600000, 0x0000000302800000, 0x0000000302800000|100%|HC| |TAMS 0x0000000302800000, 0x0000000302600000| Complete -| 16|0x0000000302800000, 0x0000000302a00000, 0x0000000302a00000|100%|HS| |TAMS 0x0000000302a00000, 0x0000000302800000| Complete -| 17|0x0000000302a00000, 0x0000000302c00000, 0x0000000302c00000|100%|HS| |TAMS 0x0000000302c00000, 0x0000000302a00000| Complete -| 18|0x0000000302c00000, 0x0000000302e00000, 0x0000000302e00000|100%|HC| |TAMS 0x0000000302e00000, 0x0000000302c00000| Complete -| 19|0x0000000302e00000, 0x0000000303000000, 0x0000000303000000|100%| O| |TAMS 0x0000000303000000, 0x0000000302e00000| Untracked -| 20|0x0000000303000000, 0x0000000303200000, 0x0000000303200000|100%| O| |TAMS 0x0000000303200000, 0x0000000303000000| Untracked -| 21|0x0000000303200000, 0x0000000303400000, 0x0000000303400000|100%| O| |TAMS 0x0000000303400000, 0x0000000303200000| Untracked -| 22|0x0000000303400000, 0x0000000303600000, 0x0000000303600000|100%| O| |TAMS 0x0000000303600000, 0x0000000303400000| Untracked -| 23|0x0000000303600000, 0x0000000303800000, 0x0000000303800000|100%| O| |TAMS 0x0000000303800000, 0x0000000303600000| Untracked -| 24|0x0000000303800000, 0x0000000303a00000, 0x0000000303a00000|100%| O| |TAMS 0x0000000303a00000, 0x0000000303800000| Untracked -| 25|0x0000000303a00000, 0x0000000303c00000, 0x0000000303c00000|100%| O| |TAMS 0x0000000303c00000, 0x0000000303a00000| Untracked -| 26|0x0000000303c00000, 0x0000000303e00000, 0x0000000303e00000|100%| O| |TAMS 0x0000000303e00000, 0x0000000303c00000| Untracked -| 27|0x0000000303e00000, 0x0000000304000000, 0x0000000304000000|100%| O| |TAMS 0x0000000304000000, 0x0000000303e00000| Untracked -| 28|0x0000000304000000, 0x0000000304200000, 0x0000000304200000|100%| O| |TAMS 0x0000000304200000, 0x0000000304000000| Untracked -| 29|0x0000000304200000, 0x0000000304400000, 0x0000000304400000|100%| O| |TAMS 0x0000000304400000, 0x0000000304200000| Untracked -| 30|0x0000000304400000, 0x0000000304600000, 0x0000000304600000|100%| O| |TAMS 0x0000000304600000, 0x0000000304400000| Untracked -| 31|0x0000000304600000, 0x0000000304800000, 0x0000000304800000|100%| O| |TAMS 0x0000000304800000, 0x0000000304600000| Untracked -| 32|0x0000000304800000, 0x0000000304a00000, 0x0000000304a00000|100%| O| |TAMS 0x0000000304a00000, 0x0000000304800000| Untracked -| 33|0x0000000304a00000, 0x0000000304c00000, 0x0000000304c00000|100%| O| |TAMS 0x0000000304c00000, 0x0000000304a00000| Untracked -| 34|0x0000000304c00000, 0x0000000304e00000, 0x0000000304e00000|100%| O| |TAMS 0x0000000304e00000, 0x0000000304c00000| Untracked -| 35|0x0000000304e00000, 0x0000000305000000, 0x0000000305000000|100%| O| |TAMS 0x0000000305000000, 0x0000000304e00000| Untracked -| 36|0x0000000305000000, 0x0000000305200000, 0x0000000305200000|100%| O| |TAMS 0x0000000305200000, 0x0000000305000000| Untracked -| 37|0x0000000305200000, 0x0000000305400000, 0x0000000305400000|100%| O| |TAMS 0x0000000305400000, 0x0000000305200000| Untracked -| 38|0x0000000305400000, 0x0000000305600000, 0x0000000305600000|100%| O| |TAMS 0x0000000305600000, 0x0000000305400000| Untracked -| 39|0x0000000305600000, 0x0000000305800000, 0x0000000305800000|100%| O| |TAMS 0x0000000305800000, 0x0000000305600000| Untracked -| 40|0x0000000305800000, 0x0000000305a00000, 0x0000000305a00000|100%| O| |TAMS 0x0000000305a00000, 0x0000000305800000| Untracked -| 41|0x0000000305a00000, 0x0000000305bbee00, 0x0000000305c00000| 87%| O| |TAMS 0x0000000305bbee00, 0x0000000305a00000| Untracked -| 42|0x0000000305c00000, 0x0000000305e00000, 0x0000000305e00000|100%|HS| |TAMS 0x0000000305c00000, 0x0000000305c00000| Complete -| 43|0x0000000305e00000, 0x0000000306000000, 0x0000000306000000|100%|HS| |TAMS 0x0000000305e00000, 0x0000000305e00000| Complete -| 44|0x0000000306000000, 0x0000000306200000, 0x0000000306200000|100%|HS| |TAMS 0x0000000306000000, 0x0000000306000000| Complete -| 45|0x0000000306200000, 0x0000000306400000, 0x0000000306400000|100%|HS| |TAMS 0x0000000306200000, 0x0000000306200000| Complete -| 46|0x0000000306400000, 0x0000000306600000, 0x0000000306600000|100%|HS| |TAMS 0x0000000306400000, 0x0000000306400000| Complete -| 47|0x0000000306600000, 0x0000000306800000, 0x0000000306800000|100%|HS| |TAMS 0x0000000306600000, 0x0000000306600000| Complete -| 48|0x0000000306800000, 0x0000000306a00000, 0x0000000306a00000|100%| O| |TAMS 0x0000000306800000, 0x0000000306800000| Untracked -| 49|0x0000000306a00000, 0x0000000306c00000, 0x0000000306c00000|100%| O| |TAMS 0x0000000306a00000, 0x0000000306a00000| Untracked -| 50|0x0000000306c00000, 0x0000000306e00000, 0x0000000306e00000|100%|HS| |TAMS 0x0000000306c00000, 0x0000000306c00000| Complete -| 51|0x0000000306e00000, 0x0000000307000000, 0x0000000307000000|100%|HS| |TAMS 0x0000000306e00000, 0x0000000306e00000| Complete -| 52|0x0000000307000000, 0x0000000307200000, 0x0000000307200000|100%|HS| |TAMS 0x0000000307000000, 0x0000000307000000| Complete -| 53|0x0000000307200000, 0x0000000307400000, 0x0000000307400000|100%|HS| |TAMS 0x0000000307200000, 0x0000000307200000| Complete -| 54|0x0000000307400000, 0x0000000307600000, 0x0000000307600000|100%| O| |TAMS 0x0000000307400000, 0x0000000307400000| Untracked -| 55|0x0000000307600000, 0x0000000307800000, 0x0000000307800000|100%| O| |TAMS 0x0000000307600000, 0x0000000307600000| Untracked -| 56|0x0000000307800000, 0x0000000307a00000, 0x0000000307a00000|100%| O| |TAMS 0x0000000307800000, 0x0000000307800000| Untracked -| 57|0x0000000307a00000, 0x0000000307c00000, 0x0000000307c00000|100%| O| |TAMS 0x0000000307a00000, 0x0000000307a00000| Untracked -| 58|0x0000000307c00000, 0x0000000307e00000, 0x0000000307e00000|100%|HS| |TAMS 0x0000000307c00000, 0x0000000307c00000| Complete -| 59|0x0000000307e00000, 0x0000000308000000, 0x0000000308000000|100%|HC| |TAMS 0x0000000307e00000, 0x0000000307e00000| Complete -| 60|0x0000000308000000, 0x0000000308200000, 0x0000000308200000|100%|HC| |TAMS 0x0000000308000000, 0x0000000308000000| Complete -| 61|0x0000000308200000, 0x0000000308200000, 0x0000000308400000| 0%| F| |TAMS 0x0000000308200000, 0x0000000308200000| Untracked -| 62|0x0000000308400000, 0x0000000308400000, 0x0000000308600000| 0%| F| |TAMS 0x0000000308400000, 0x0000000308400000| Untracked -| 63|0x0000000308600000, 0x0000000308600000, 0x0000000308800000| 0%| F| |TAMS 0x0000000308600000, 0x0000000308600000| Untracked -| 64|0x0000000308800000, 0x0000000308800000, 0x0000000308a00000| 0%| F| |TAMS 0x0000000308800000, 0x0000000308800000| Untracked -| 65|0x0000000308a00000, 0x0000000308a00000, 0x0000000308c00000| 0%| F| |TAMS 0x0000000308a00000, 0x0000000308a00000| Untracked -| 66|0x0000000308c00000, 0x0000000308c00000, 0x0000000308e00000| 0%| F| |TAMS 0x0000000308c00000, 0x0000000308c00000| Untracked -| 67|0x0000000308e00000, 0x0000000308e00000, 0x0000000309000000| 0%| F| |TAMS 0x0000000308e00000, 0x0000000308e00000| Untracked -| 68|0x0000000309000000, 0x0000000309000000, 0x0000000309200000| 0%| F| |TAMS 0x0000000309000000, 0x0000000309000000| Untracked -| 69|0x0000000309200000, 0x0000000309200000, 0x0000000309400000| 0%| F| |TAMS 0x0000000309200000, 0x0000000309200000| Untracked -| 70|0x0000000309400000, 0x000000030957ce98, 0x0000000309600000| 74%| S|CS|TAMS 0x0000000309400000, 0x0000000309400000| Complete -| 71|0x0000000309600000, 0x0000000309800000, 0x0000000309800000|100%| O| |TAMS 0x0000000309600000, 0x0000000309600000| Untracked -| 72|0x0000000309800000, 0x0000000309a00000, 0x0000000309a00000|100%| O| |TAMS 0x0000000309800000, 0x0000000309800000| Untracked -| 73|0x0000000309a00000, 0x0000000309c00000, 0x0000000309c00000|100%| O| |TAMS 0x0000000309a00000, 0x0000000309a00000| Untracked -| 74|0x0000000309c00000, 0x0000000309e00000, 0x0000000309e00000|100%| O| |TAMS 0x0000000309c00000, 0x0000000309c00000| Untracked -| 75|0x0000000309e00000, 0x000000030a000000, 0x000000030a000000|100%| O| |TAMS 0x0000000309e00000, 0x0000000309e00000| Untracked -| 76|0x000000030a000000, 0x000000030a200000, 0x000000030a200000|100%| O| |TAMS 0x000000030a000000, 0x000000030a000000| Untracked -| 77|0x000000030a200000, 0x000000030a400000, 0x000000030a400000|100%| O| |TAMS 0x000000030a200000, 0x000000030a200000| Untracked -| 78|0x000000030a400000, 0x000000030a600000, 0x000000030a600000|100%| O| |TAMS 0x000000030a400000, 0x000000030a400000| Untracked -| 79|0x000000030a600000, 0x000000030a741e00, 0x000000030a800000| 62%| O| |TAMS 0x000000030a600000, 0x000000030a600000| Untracked -| 80|0x000000030a800000, 0x000000030a800000, 0x000000030aa00000| 0%| F| |TAMS 0x000000030a800000, 0x000000030a800000| Untracked -| 81|0x000000030aa00000, 0x000000030aa00000, 0x000000030ac00000| 0%| F| |TAMS 0x000000030aa00000, 0x000000030aa00000| Untracked -| 82|0x000000030ac00000, 0x000000030ae00000, 0x000000030ae00000|100%| S|CS|TAMS 0x000000030ac00000, 0x000000030ac00000| Complete -| 83|0x000000030ae00000, 0x000000030ae00000, 0x000000030b000000| 0%| F| |TAMS 0x000000030ae00000, 0x000000030ae00000| Untracked -| 84|0x000000030b000000, 0x000000030b000000, 0x000000030b200000| 0%| F| |TAMS 0x000000030b000000, 0x000000030b000000| Untracked -| 85|0x000000030b200000, 0x000000030b400000, 0x000000030b400000|100%| S|CS|TAMS 0x000000030b200000, 0x000000030b200000| Complete -| 86|0x000000030b400000, 0x000000030b600000, 0x000000030b600000|100%| S|CS|TAMS 0x000000030b400000, 0x000000030b400000| Complete -| 87|0x000000030b600000, 0x000000030b800000, 0x000000030b800000|100%| S|CS|TAMS 0x000000030b600000, 0x000000030b600000| Complete -| 88|0x000000030b800000, 0x000000030b800000, 0x000000030ba00000| 0%| F| |TAMS 0x000000030b800000, 0x000000030b800000| Untracked -| 89|0x000000030ba00000, 0x000000030ba00000, 0x000000030bc00000| 0%| F| |TAMS 0x000000030ba00000, 0x000000030ba00000| Untracked -| 90|0x000000030bc00000, 0x000000030bc00000, 0x000000030be00000| 0%| F| |TAMS 0x000000030bc00000, 0x000000030bc00000| Untracked -| 91|0x000000030be00000, 0x000000030be00000, 0x000000030c000000| 0%| F| |TAMS 0x000000030be00000, 0x000000030be00000| Untracked -| 92|0x000000030c000000, 0x000000030c000000, 0x000000030c200000| 0%| F| |TAMS 0x000000030c000000, 0x000000030c000000| Untracked -| 93|0x000000030c200000, 0x000000030c200000, 0x000000030c400000| 0%| F| |TAMS 0x000000030c200000, 0x000000030c200000| Untracked -| 94|0x000000030c400000, 0x000000030c400000, 0x000000030c600000| 0%| F| |TAMS 0x000000030c400000, 0x000000030c400000| Untracked -| 95|0x000000030c600000, 0x000000030c600000, 0x000000030c800000| 0%| F| |TAMS 0x000000030c600000, 0x000000030c600000| Untracked -| 96|0x000000030c800000, 0x000000030c800000, 0x000000030ca00000| 0%| F| |TAMS 0x000000030c800000, 0x000000030c800000| Untracked -| 97|0x000000030ca00000, 0x000000030ca00000, 0x000000030cc00000| 0%| F| |TAMS 0x000000030ca00000, 0x000000030ca00000| Untracked -| 98|0x000000030cc00000, 0x000000030cc00000, 0x000000030ce00000| 0%| F| |TAMS 0x000000030cc00000, 0x000000030cc00000| Untracked -| 99|0x000000030ce00000, 0x000000030ce00000, 0x000000030d000000| 0%| F| |TAMS 0x000000030ce00000, 0x000000030ce00000| Untracked -| 100|0x000000030d000000, 0x000000030d000000, 0x000000030d200000| 0%| F| |TAMS 0x000000030d000000, 0x000000030d000000| Untracked -| 101|0x000000030d200000, 0x000000030d200000, 0x000000030d400000| 0%| F| |TAMS 0x000000030d200000, 0x000000030d200000| Untracked -| 102|0x000000030d400000, 0x000000030d400000, 0x000000030d600000| 0%| F| |TAMS 0x000000030d400000, 0x000000030d400000| Untracked -| 103|0x000000030d600000, 0x000000030d600000, 0x000000030d800000| 0%| F| |TAMS 0x000000030d600000, 0x000000030d600000| Untracked -| 104|0x000000030d800000, 0x000000030d800000, 0x000000030da00000| 0%| F| |TAMS 0x000000030d800000, 0x000000030d800000| Untracked -| 105|0x000000030da00000, 0x000000030da00000, 0x000000030dc00000| 0%| F| |TAMS 0x000000030da00000, 0x000000030da00000| Untracked -| 106|0x000000030dc00000, 0x000000030dc00000, 0x000000030de00000| 0%| F| |TAMS 0x000000030dc00000, 0x000000030dc00000| Untracked -| 107|0x000000030de00000, 0x000000030de00000, 0x000000030e000000| 0%| F| |TAMS 0x000000030de00000, 0x000000030de00000| Untracked -| 108|0x000000030e000000, 0x000000030e000000, 0x000000030e200000| 0%| F| |TAMS 0x000000030e000000, 0x000000030e000000| Untracked -| 109|0x000000030e200000, 0x000000030e200000, 0x000000030e400000| 0%| F| |TAMS 0x000000030e200000, 0x000000030e200000| Untracked -| 110|0x000000030e400000, 0x000000030e400000, 0x000000030e600000| 0%| F| |TAMS 0x000000030e400000, 0x000000030e400000| Untracked -| 111|0x000000030e600000, 0x000000030e600000, 0x000000030e800000| 0%| F| |TAMS 0x000000030e600000, 0x000000030e600000| Untracked -| 112|0x000000030e800000, 0x000000030e800000, 0x000000030ea00000| 0%| F| |TAMS 0x000000030e800000, 0x000000030e800000| Untracked -| 113|0x000000030ea00000, 0x000000030ea00000, 0x000000030ec00000| 0%| F| |TAMS 0x000000030ea00000, 0x000000030ea00000| Untracked -| 114|0x000000030ec00000, 0x000000030ec00000, 0x000000030ee00000| 0%| F| |TAMS 0x000000030ec00000, 0x000000030ec00000| Untracked -| 115|0x000000030ee00000, 0x000000030ee00000, 0x000000030f000000| 0%| F| |TAMS 0x000000030ee00000, 0x000000030ee00000| Untracked -| 116|0x000000030f000000, 0x000000030f000000, 0x000000030f200000| 0%| F| |TAMS 0x000000030f000000, 0x000000030f000000| Untracked -| 117|0x000000030f200000, 0x000000030f200000, 0x000000030f400000| 0%| F| |TAMS 0x000000030f200000, 0x000000030f200000| Untracked -| 118|0x000000030f400000, 0x000000030f400000, 0x000000030f600000| 0%| F| |TAMS 0x000000030f400000, 0x000000030f400000| Untracked -| 119|0x000000030f600000, 0x000000030f722fa0, 0x000000030f800000| 56%| E| |TAMS 0x000000030f600000, 0x000000030f600000| Complete -| 120|0x000000030f800000, 0x000000030fa00000, 0x000000030fa00000|100%| E|CS|TAMS 0x000000030f800000, 0x000000030f800000| Complete -| 121|0x000000030fa00000, 0x000000030fc00000, 0x000000030fc00000|100%| E|CS|TAMS 0x000000030fa00000, 0x000000030fa00000| Complete -| 122|0x000000030fc00000, 0x000000030fe00000, 0x000000030fe00000|100%| E|CS|TAMS 0x000000030fc00000, 0x000000030fc00000| Complete -| 123|0x000000030fe00000, 0x0000000310000000, 0x0000000310000000|100%| E|CS|TAMS 0x000000030fe00000, 0x000000030fe00000| Complete -| 124|0x0000000310000000, 0x0000000310200000, 0x0000000310200000|100%| E|CS|TAMS 0x0000000310000000, 0x0000000310000000| Complete -| 125|0x0000000310200000, 0x0000000310400000, 0x0000000310400000|100%| E|CS|TAMS 0x0000000310200000, 0x0000000310200000| Complete -| 126|0x0000000310400000, 0x0000000310600000, 0x0000000310600000|100%| E|CS|TAMS 0x0000000310400000, 0x0000000310400000| Complete -| 127|0x0000000310600000, 0x0000000310800000, 0x0000000310800000|100%| E|CS|TAMS 0x0000000310600000, 0x0000000310600000| Complete -| 128|0x0000000310800000, 0x0000000310a00000, 0x0000000310a00000|100%| E|CS|TAMS 0x0000000310800000, 0x0000000310800000| Complete -| 129|0x0000000310a00000, 0x0000000310c00000, 0x0000000310c00000|100%| E|CS|TAMS 0x0000000310a00000, 0x0000000310a00000| Complete -| 130|0x0000000310c00000, 0x0000000310e00000, 0x0000000310e00000|100%| E|CS|TAMS 0x0000000310c00000, 0x0000000310c00000| Complete -| 131|0x0000000310e00000, 0x0000000311000000, 0x0000000311000000|100%| E|CS|TAMS 0x0000000310e00000, 0x0000000310e00000| Complete -| 132|0x0000000311000000, 0x0000000311200000, 0x0000000311200000|100%| E|CS|TAMS 0x0000000311000000, 0x0000000311000000| Complete -| 133|0x0000000311200000, 0x0000000311400000, 0x0000000311400000|100%| E|CS|TAMS 0x0000000311200000, 0x0000000311200000| Complete -| 134|0x0000000311400000, 0x0000000311600000, 0x0000000311600000|100%| E|CS|TAMS 0x0000000311400000, 0x0000000311400000| Complete -| 135|0x0000000311600000, 0x0000000311800000, 0x0000000311800000|100%| E|CS|TAMS 0x0000000311600000, 0x0000000311600000| Complete -| 136|0x0000000311800000, 0x0000000311a00000, 0x0000000311a00000|100%| E|CS|TAMS 0x0000000311800000, 0x0000000311800000| Complete -| 137|0x0000000311a00000, 0x0000000311c00000, 0x0000000311c00000|100%| E|CS|TAMS 0x0000000311a00000, 0x0000000311a00000| Complete -| 138|0x0000000311c00000, 0x0000000311e00000, 0x0000000311e00000|100%| E|CS|TAMS 0x0000000311c00000, 0x0000000311c00000| Complete -| 139|0x0000000311e00000, 0x0000000312000000, 0x0000000312000000|100%| E|CS|TAMS 0x0000000311e00000, 0x0000000311e00000| Complete -| 140|0x0000000312000000, 0x0000000312200000, 0x0000000312200000|100%| E|CS|TAMS 0x0000000312000000, 0x0000000312000000| Complete -| 141|0x0000000312200000, 0x0000000312400000, 0x0000000312400000|100%| E|CS|TAMS 0x0000000312200000, 0x0000000312200000| Complete -| 142|0x0000000312400000, 0x0000000312600000, 0x0000000312600000|100%| E|CS|TAMS 0x0000000312400000, 0x0000000312400000| Complete -| 143|0x0000000312600000, 0x0000000312800000, 0x0000000312800000|100%| E|CS|TAMS 0x0000000312600000, 0x0000000312600000| Complete -| 144|0x0000000312800000, 0x0000000312a00000, 0x0000000312a00000|100%| E|CS|TAMS 0x0000000312800000, 0x0000000312800000| Complete -| 145|0x0000000312a00000, 0x0000000312c00000, 0x0000000312c00000|100%| E|CS|TAMS 0x0000000312a00000, 0x0000000312a00000| Complete -| 146|0x0000000312c00000, 0x0000000312e00000, 0x0000000312e00000|100%| E|CS|TAMS 0x0000000312c00000, 0x0000000312c00000| Complete -| 147|0x0000000312e00000, 0x0000000313000000, 0x0000000313000000|100%| E|CS|TAMS 0x0000000312e00000, 0x0000000312e00000| Complete -| 148|0x0000000313000000, 0x0000000313200000, 0x0000000313200000|100%| E|CS|TAMS 0x0000000313000000, 0x0000000313000000| Complete -| 149|0x0000000313200000, 0x0000000313400000, 0x0000000313400000|100%| E|CS|TAMS 0x0000000313200000, 0x0000000313200000| Complete -| 150|0x0000000313400000, 0x0000000313600000, 0x0000000313600000|100%| E|CS|TAMS 0x0000000313400000, 0x0000000313400000| Complete -| 151|0x0000000313600000, 0x0000000313800000, 0x0000000313800000|100%| E|CS|TAMS 0x0000000313600000, 0x0000000313600000| Complete -| 152|0x0000000313800000, 0x0000000313a00000, 0x0000000313a00000|100%| E|CS|TAMS 0x0000000313800000, 0x0000000313800000| Complete -| 153|0x0000000313a00000, 0x0000000313c00000, 0x0000000313c00000|100%| E|CS|TAMS 0x0000000313a00000, 0x0000000313a00000| Complete -| 154|0x0000000313c00000, 0x0000000313e00000, 0x0000000313e00000|100%| E|CS|TAMS 0x0000000313c00000, 0x0000000313c00000| Complete -| 155|0x0000000313e00000, 0x0000000314000000, 0x0000000314000000|100%| E|CS|TAMS 0x0000000313e00000, 0x0000000313e00000| Complete -| 156|0x0000000314000000, 0x0000000314200000, 0x0000000314200000|100%| E|CS|TAMS 0x0000000314000000, 0x0000000314000000| Complete -| 157|0x0000000314200000, 0x0000000314400000, 0x0000000314400000|100%| E|CS|TAMS 0x0000000314200000, 0x0000000314200000| Complete -| 158|0x0000000314400000, 0x0000000314600000, 0x0000000314600000|100%| E|CS|TAMS 0x0000000314400000, 0x0000000314400000| Complete -| 340|0x000000032b000000, 0x000000032b200000, 0x000000032b200000|100%| E|CS|TAMS 0x000000032b000000, 0x000000032b000000| Complete -| 341|0x000000032b200000, 0x000000032b400000, 0x000000032b400000|100%| E|CS|TAMS 0x000000032b200000, 0x000000032b200000| Complete -| 440|0x0000000337800000, 0x0000000337a00000, 0x0000000337a00000|100%| E|CS|TAMS 0x0000000337800000, 0x0000000337800000| Complete -| 441|0x0000000337a00000, 0x0000000337c00000, 0x0000000337c00000|100%| E|CS|TAMS 0x0000000337a00000, 0x0000000337a00000| Complete -| 442|0x0000000337c00000, 0x0000000337e00000, 0x0000000337e00000|100%| E|CS|TAMS 0x0000000337c00000, 0x0000000337c00000| Complete -| 767|0x0000000360600000, 0x0000000360800000, 0x0000000360800000|100%| E|CS|TAMS 0x0000000360600000, 0x0000000360600000| Complete -|2046|0x0000000400400000, 0x0000000400574000, 0x0000000400600000| 72%|OA| |TAMS 0x0000000400574000, 0x0000000400400000| Untracked -|2047|0x0000000400600000, 0x0000000400674000, 0x0000000400800000| 22%|CA| |TAMS 0x0000000400674000, 0x0000000400600000| Untracked - -Card table byte_map: [0x00000001225b8000,0x0000000122db8000] _byte_map_base: 0x0000000120db4000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x00000008c9094010, (CMBitMap*) 0x00000008c9094050 - Prev Bits: [0x00000001235b8000, 0x00000001275b8000) - Next Bits: [0x00000001275b8000, 0x000000012b5b8000) - -Polling page: 0x0000000104efc000 - -Metaspace: - -Usage: - Non-class: 130.04 MB used. - Class: 17.77 MB used. - Both: 147.81 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 130.75 MB ( 68%) committed, 3 nodes. - Class space: 1.00 GB reserved, 18.31 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 149.06 MB ( 12%) committed. - -Chunk freelists: - Non-Class: 13.06 MB - Class: 13.70 MB - Both: 26.76 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 164.81 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 14. -num_arena_births: 2064. -num_arena_deaths: 0. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2382. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 14. -num_chunks_taken_from_freelist: 7764. -num_chunk_merges: 11. -num_chunk_splits: 5785. -num_chunks_enlarged: 4384. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=9399Kb max_used=9399Kb free=109769Kb - bounds [0x000000011b158000, 0x000000011ba88000, 0x00000001225b8000] -CodeHeap 'profiled nmethods': size=119152Kb used=19479Kb max_used=19791Kb free=99672Kb - bounds [0x00000001135b8000, 0x0000000114918000, 0x000000011aa14000] -CodeHeap 'non-nmethods': size=7440Kb used=3323Kb max_used=3484Kb free=4116Kb - bounds [0x000000011aa14000, 0x000000011ad94000, 0x000000011b158000] - total_blobs=11894 nmethods=10833 adapters=973 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 20.875 Thread 0x00000008cc043600 14786 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) -Event: 20.875 Thread 0x00000008cc178000 14787 3 com.google.protobuf.MapEntry::writeTo (26 bytes) -Event: 20.875 Thread 0x00000008cc043600 nmethod 14786 0x000000011489fa90 code [0x000000011489fc80, 0x000000011489fee8] -Event: 20.875 Thread 0x00000008cc178000 nmethod 14787 0x000000011443a110 code [0x000000011443a2c0, 0x000000011443a4b8] -Event: 20.875 Thread 0x00000008cc042400 nmethod 14781 0x0000000114439190 code [0x0000000114439440, 0x0000000114439cf8] -Event: 20.875 Thread 0x00000008cc042400 14790 3 org.apache.comet.ConfigEntry::key (13 bytes) -Event: 20.875 Thread 0x00000008cc043600 14789 3 scala.collection.convert.Wrappers$IteratorWrapper::$jacocoInit (52 bytes) -Event: 20.875 Thread 0x00000008cc178000 14791 3 scala.collection.immutable.Map$EmptyMap$::updated (21 bytes) -Event: 20.875 Thread 0x00000008cc042400 nmethod 14790 0x0000000114438d10 code [0x0000000114438ec0, 0x0000000114439048] -Event: 20.875 Thread 0x00000008cc042400 14792 3 java.util.GregorianCalendar::computeFields (85 bytes) -Event: 20.875 Thread 0x00000008d11a4c00 nmethod 14782 0x0000000113a75490 code [0x0000000113a75800, 0x0000000113a76b58] -Event: 20.875 Thread 0x00000008cc178000 nmethod 14791 0x0000000114438790 code [0x0000000114438940, 0x0000000114438b78] -Event: 20.875 Thread 0x00000008cc178000 14788 3 scala.collection.immutable.List::apply (15 bytes) -Event: 20.875 Thread 0x00000008cc178000 nmethod 14788 0x0000000113a74f90 code [0x0000000113a75140, 0x0000000113a75338] -Event: 20.875 Thread 0x00000008d11a4c00 14793 3 org.apache.comet.serde.Config$ConfigMap$Builder::internalGetMutableEntries (110 bytes) -Event: 20.875 Thread 0x00000008cc178000 14794 3 org.apache.comet.serde.Config$ConfigMap$Builder::putEntries (87 bytes) -Event: 20.875 Thread 0x00000008cc042400 nmethod 14792 0x0000000113a74790 code [0x0000000113a749c0, 0x0000000113a74e18] -Event: 20.875 Thread 0x00000008cc043600 nmethod 14789 0x0000000113a73910 code [0x0000000113a73b80, 0x0000000113a74538] -Event: 20.875 Thread 0x00000008d11a4c00 nmethod 14793 0x0000000113a72f90 code [0x0000000113a731c0, 0x0000000113a73758] -Event: 20.875 Thread 0x00000008cc178000 nmethod 14794 0x0000000113a72510 code [0x0000000113a72740, 0x0000000113a72cc8] - -GC Heap History (20 events): -Event: 6.523 GC heap before -{Heap before GC invocations=25 (full 0): - garbage-first heap total 342016K, used 283317K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 85 young (174080K), 13 survivors (26624K) - Metaspace used 123782K, committed 124608K, reserved 1179648K - class space used 14869K, committed 15296K, reserved 1048576K -} -Event: 6.526 GC heap after -{Heap after GC invocations=26 (full 0): - garbage-first heap total 342016K, used 148707K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 6 young (12288K), 6 survivors (12288K) - Metaspace used 123782K, committed 124608K, reserved 1179648K - class space used 14869K, committed 15296K, reserved 1048576K -} -Event: 6.818 GC heap before -{Heap before GC invocations=26 (full 0): - garbage-first heap total 342016K, used 277731K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 69 young (141312K), 6 survivors (12288K) - Metaspace used 127657K, committed 128512K, reserved 1179648K - class space used 15121K, committed 15488K, reserved 1048576K -} -Event: 6.819 GC heap after -{Heap after GC invocations=27 (full 0): - garbage-first heap total 342016K, used 147405K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 127657K, committed 128512K, reserved 1179648K - class space used 15121K, committed 15488K, reserved 1048576K -} -Event: 7.010 GC heap before -{Heap before GC invocations=27 (full 0): - garbage-first heap total 342016K, used 290765K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 70 young (143360K), 5 survivors (10240K) - Metaspace used 133183K, committed 134080K, reserved 1179648K - class space used 15911K, committed 16320K, reserved 1048576K -} -Event: 7.011 GC heap after -{Heap after GC invocations=28 (full 0): - garbage-first heap total 342016K, used 133772K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 133183K, committed 134080K, reserved 1179648K - class space used 15911K, committed 16320K, reserved 1048576K -} -Event: 7.271 GC heap before -{Heap before GC invocations=28 (full 0): - garbage-first heap total 342016K, used 291468K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 76 young (155648K), 3 survivors (6144K) - Metaspace used 140639K, committed 141632K, reserved 1179648K - class space used 16853K, committed 17280K, reserved 1048576K -} -Event: 7.273 GC heap after -{Heap after GC invocations=29 (full 0): - garbage-first heap total 342016K, used 135270K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 140639K, committed 141632K, reserved 1179648K - class space used 16853K, committed 17280K, reserved 1048576K -} -Event: 7.490 GC heap before -{Heap before GC invocations=29 (full 0): - garbage-first heap total 342016K, used 288870K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 77 young (157696K), 4 survivors (8192K) - Metaspace used 148294K, committed 149376K, reserved 1245184K - class space used 17691K, committed 18176K, reserved 1048576K -} -Event: 7.492 GC heap after -{Heap after GC invocations=30 (full 0): - garbage-first heap total 342016K, used 137462K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148294K, committed 149376K, reserved 1245184K - class space used 17691K, committed 18176K, reserved 1048576K -} -Event: 7.850 GC heap before -{Heap before GC invocations=30 (full 0): - garbage-first heap total 342016K, used 286966K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 78 young (159744K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 7.853 GC heap after -{Heap after GC invocations=31 (full 0): - garbage-first heap total 342016K, used 137868K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.168 GC heap before -{Heap before GC invocations=31 (full 0): - garbage-first heap total 342016K, used 291468K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 80 young (163840K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.170 GC heap after -{Heap after GC invocations=32 (full 0): - garbage-first heap total 342016K, used 137599K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.482 GC heap before -{Heap before GC invocations=32 (full 0): - garbage-first heap total 342016K, used 293247K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 81 young (165888K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.484 GC heap after -{Heap after GC invocations=33 (full 0): - garbage-first heap total 342016K, used 137844K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.800 GC heap before -{Heap before GC invocations=33 (full 0): - garbage-first heap total 342016K, used 293492K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 81 young (165888K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 8.802 GC heap after -{Heap after GC invocations=34 (full 0): - garbage-first heap total 342016K, used 137466K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 9.102 GC heap before -{Heap before GC invocations=34 (full 0): - garbage-first heap total 342016K, used 293114K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 81 young (165888K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} -Event: 9.104 GC heap after -{Heap after GC invocations=35 (full 0): - garbage-first heap total 342016K, used 137622K [0x0000000300800000, 0x0000000400800000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 148795K, committed 149888K, reserved 1245184K - class space used 17760K, committed 18240K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.397 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li -Event: 3.397 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.397 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.362 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-16636361560025204640.dylib -Event: 5.872 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 5.872 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 5.872 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 5.872 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 5.872 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 5.872 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 5.872 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.872 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 5.873 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 5.873 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 6.021 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15419921919615002278.dylib -Event: 6.735 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-cd884805-d66d-4abd-b6de-6004019a551b-libsnappyjava.dylib - -Deoptimization events (20 events): -Event: 20.871 Thread 0x00000008d513de00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 -Event: 20.871 Thread 0x00000008d513f000 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 -Event: 20.871 Thread 0x00000008d513e400 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 -Event: 20.871 Thread 0x00000008d513ea00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 -Event: 20.871 Thread 0x00000008d513e400 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 -Event: 20.871 Thread 0x00000008d513e400 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c5312e0 -Event: 20.871 Thread 0x00000008d513de00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 -Event: 20.871 Thread 0x00000008d513e400 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c531290 mode 2 -Event: 20.871 Thread 0x00000008d513de00 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c1252e0 -Event: 20.871 Thread 0x00000008d513de00 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c125290 mode 2 -Event: 20.871 Thread 0x00000008d513d800 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b623808 relative=0x0000000000000288 -Event: 20.871 Thread 0x00000008d513f000 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 -Event: 20.871 Thread 0x00000008d513f000 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017cd492e0 -Event: 20.871 Thread 0x00000008d513d800 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 -Event: 20.871 Thread 0x00000008d513ea00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b623808 method=java.io.SequenceInputStream.peekNextStream()V @ 4 c2 -Event: 20.871 Thread 0x00000008d513d800 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017bccd2e0 -Event: 20.871 Thread 0x00000008d513f000 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017cd49290 mode 2 -Event: 20.871 Thread 0x00000008d513ea00 DEOPT PACKING pc=0x000000011b623808 sp=0x000000017c93d2e0 -Event: 20.871 Thread 0x00000008d513d800 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017bccd290 mode 2 -Event: 20.871 Thread 0x00000008d513ea00 DEOPT UNPACKING pc=0x000000011aa5b91c sp=0x000000017c93d290 mode 2 - -Classes loaded (20 events): -Event: 7.424 Loading class java/lang/Class$EnclosingMethodInfo -Event: 7.424 Loading class java/lang/Class$EnclosingMethodInfo done -Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper -Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$Unmapper -Event: 15.931 Loading class jdk/internal/access/foreign/UnmapperProxy -Event: 15.931 Loading class jdk/internal/access/foreign/UnmapperProxy done -Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$Unmapper done -Event: 15.931 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done -Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask -Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask -Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done -Event: 20.497 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done -Event: 20.497 Loading class java/util/function/LongBinaryOperator -Event: 20.497 Loading class java/util/function/LongBinaryOperator done -Event: 20.725 Loading class sun/nio/ch/ChannelInputStream -Event: 20.725 Loading class sun/nio/ch/ChannelInputStream done -Event: 20.810 Loading class java/math/BigDecimal$StringBuilderHelper -Event: 20.810 Loading class java/math/BigDecimal$StringBuilderHelper done -Event: 20.871 Loading class java/nio/channels/Channels$ReadableByteChannelImpl -Event: 20.872 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 20.867 Thread 0x00000008d513f000 Exception (0x000000030fd89a68) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513de00 Exception (0x000000030fc51c08) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513ea00 Exception (0x0000000310280458) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513d800 Exception (0x000000030fa2d460) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513e400 Exception (0x000000030fdcf718) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513f000 Exception (0x000000030fd8d080) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.867 Thread 0x00000008d513de00 Exception (0x000000030fc55220) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb62968) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fda1250) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513ea00 Exception (0x00000003102955a0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513d800 Exception (0x000000030fa7a188) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb66958) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513e400 Exception (0x000000030fddfed0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fda5028) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513ea00 Exception (0x0000000310299378) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513de00 Exception (0x000000030fb6a238) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513f000 Exception (0x000000030fbb2798) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513ea00 Exception (0x000000030f802738) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513d800 Exception (0x000000030fa7dbe8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.869 Thread 0x00000008d513e400 Exception (0x000000030fde3930) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] - -VM Operations (20 events): -Event: 7.573 Executing VM operation: ICBufferFull -Event: 7.573 Executing VM operation: ICBufferFull done -Event: 7.850 Executing VM operation: G1CollectForAllocation -Event: 7.853 Executing VM operation: G1CollectForAllocation done -Event: 8.168 Executing VM operation: G1CollectForAllocation -Event: 8.170 Executing VM operation: G1CollectForAllocation done -Event: 8.482 Executing VM operation: G1CollectForAllocation -Event: 8.484 Executing VM operation: G1CollectForAllocation done -Event: 8.800 Executing VM operation: G1CollectForAllocation -Event: 8.802 Executing VM operation: G1CollectForAllocation done -Event: 9.102 Executing VM operation: G1CollectForAllocation -Event: 9.104 Executing VM operation: G1CollectForAllocation done -Event: 12.119 Executing VM operation: Cleanup -Event: 12.119 Executing VM operation: Cleanup done -Event: 16.139 Executing VM operation: Cleanup -Event: 16.139 Executing VM operation: Cleanup done -Event: 20.799 Executing VM operation: ICBufferFull -Event: 20.799 Executing VM operation: ICBufferFull done -Event: 20.833 Executing VM operation: ICBufferFull -Event: 20.833 Executing VM operation: ICBufferFull done - -Memory protections (20 events): -Event: 7.055 Protecting memory [0x000000017e010000,0x000000017e01c000] with protection modes 3 -Event: 7.279 Protecting memory [0x000000017e010000,0x000000017e01c000] with protection modes 0 -Event: 7.281 Protecting memory [0x000000017e41c000,0x000000017e428000] with protection modes 0 -Event: 7.281 Protecting memory [0x000000017e828000,0x000000017e834000] with protection modes 0 -Event: 7.281 Protecting memory [0x000000017ec34000,0x000000017ec40000] with protection modes 0 -Event: 7.281 Protecting memory [0x000000017f040000,0x000000017f04c000] with protection modes 0 -Event: 9.190 Protecting memory [0x00000001720ec000,0x00000001720f8000] with protection modes 3 -Event: 9.190 Protecting memory [0x000000016e39c000,0x000000016e3a8000] with protection modes 3 -Event: 9.190 Protecting memory [0x000000016f3f0000,0x000000016f3fc000] with protection modes 3 -Event: 9.192 Protecting memory [0x000000016e190000,0x000000016e19c000] with protection modes 3 -Event: 9.192 Protecting memory [0x000000016ebcc000,0x000000016ebd8000] with protection modes 3 -Event: 9.203 Protecting memory [0x000000016e9c0000,0x000000016e9cc000] with protection modes 3 -Event: 9.203 Protecting memory [0x000000016e7b4000,0x000000016e7c0000] with protection modes 3 -Event: 14.208 Protecting memory [0x000000016e5a8000,0x000000016e5b4000] with protection modes 3 -Event: 20.727 Protecting memory [0x000000016e190000,0x000000016e19c000] with protection modes 0 -Event: 20.747 Protecting memory [0x000000016e39c000,0x000000016e3a8000] with protection modes 0 -Event: 20.747 Protecting memory [0x000000016e5a8000,0x000000016e5b4000] with protection modes 0 -Event: 20.747 Protecting memory [0x000000016e7b4000,0x000000016e7c0000] with protection modes 0 -Event: 20.748 Protecting memory [0x000000016e9c0000,0x000000016e9cc000] with protection modes 0 -Event: 20.801 Protecting memory [0x00000004115d8000,0x00000004115e4000] with protection modes 0 - -Nmethod flushes (20 events): -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114889d90 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011488ac10 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114894790 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114894c10 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114895a90 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114896190 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114897790 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114898610 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x0000000114899490 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489a210 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489ad10 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489d490 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489fa90 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x000000011489ff10 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a3c10 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a4090 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a4610 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a6790 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a6c90 -Event: 7.572 Thread 0x00000008cc042a00 flushing nmethod 0x00000001148a7a90 - -Events (20 events): -Event: 7.055 Thread 0x00000008d11a4600 Thread exited: 0x00000008d11a4600 -Event: 7.279 Thread 0x00000008d4d00000 Thread added: 0x00000008d708ea00 -Event: 7.281 Thread 0x00000008d708ea00 Thread added: 0x00000008d708f000 -Event: 7.281 Thread 0x00000008d708ea00 Thread added: 0x00000008d708f600 -Event: 7.281 Thread 0x00000008d708f600 Thread added: 0x00000008d70e4000 -Event: 7.281 Thread 0x00000008d70e4000 Thread added: 0x00000008d70e4600 -Event: 9.190 Thread 0x00000008d513f600 Thread exited: 0x00000008d513f600 -Event: 9.190 Thread 0x00000008d11a4c00 Thread exited: 0x00000008d11a4c00 -Event: 9.190 Thread 0x00000008cc178000 Thread exited: 0x00000008cc178000 -Event: 9.192 Thread 0x00000008cc043600 Thread exited: 0x00000008cc043600 -Event: 9.192 Thread 0x00000008d4bfc600 Thread exited: 0x00000008d4bfc600 -Event: 9.203 Thread 0x00000008d4bfc000 Thread exited: 0x00000008d4bfc000 -Event: 9.203 Thread 0x00000008cc179800 Thread exited: 0x00000008cc179800 -Event: 14.208 Thread 0x00000008d68d7600 Thread exited: 0x00000008d68d7600 -Event: 20.727 Thread 0x00000008cc042400 Thread added: 0x00000008d4bfc600 -Event: 20.747 Thread 0x00000008cc042400 Thread added: 0x00000008cc043600 -Event: 20.747 Thread 0x00000008cc043600 Thread added: 0x00000008cc179800 -Event: 20.747 Thread 0x00000008cc043600 Thread added: 0x00000008cc178000 -Event: 20.748 Thread 0x00000008cc043600 Thread added: 0x00000008d11a4c00 -Event: 20.801 Thread 0x00000008d4d14600 Thread added: 0x00000008d70e7000 - - -Dynamic libraries: -0x0000000104f14000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x000000010aa74000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x0000000104f3c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x0000000104f6c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x0000000104fbc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x000000010a478000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x000000010513c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x000000010a4c4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x000000010a4a0000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x000000010a568000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x000000010a57c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x000000010a590000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x000000010a5a4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000400800000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-16636361560025204640.dylib -0x000000010a67c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15419921919615002278.dylib -0x000000010a650000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-cd884805-d66d-4abd-b6de-6004019a551b-libsnappyjava.dylib - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:39 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 10.03 7.52 5.24 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(1442992k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. diff --git a/spark/hs_err_pid59058.log b/spark/hs_err_pid59058.log deleted file mode 100644 index 8df2ae515e..0000000000 --- a/spark/hs_err_pid59058.log +++ /dev/null @@ -1,1568 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x000000010a7c1530, pid=59058, tid=63499 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 08:58:03 2026 MDT elapsed time: 20.974203 seconds (0d 0h 0m 20s) - ---------------- T H R E A D --------------- - -Current thread (0x0000000b661e1200): JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=63499, stack(0x000000017c1b0000,0x000000017c5b3000)] - -Stack: [0x000000017c1b0000,0x000000017c5b3000], sp=0x000000017c5aac00, free space=4075k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 -C [libcomet-14395648754114117926.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 -C [libcomet-14395648754114117926.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc -C [libcomet-14395648754114117926.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 -C [libcomet-14395648754114117926.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 -C [libcomet-14395648754114117926.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c -C [libcomet-14395648754114117926.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 -C [libcomet-14395648754114117926.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 -C [libcomet-14395648754114117926.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 -C [libcomet-14395648754114117926.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 -C [libcomet-14395648754114117926.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 -C [libcomet-14395648754114117926.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 -C [libcomet-14395648754114117926.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 -C [libcomet-14395648754114117926.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c -C [libcomet-14395648754114117926.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 -C [libcomet-14395648754114117926.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 -C [libcomet-14395648754114117926.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c -C [libcomet-14395648754114117926.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 -C [libcomet-14395648754114117926.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 -C [libcomet-14395648754114117926.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 -C [libcomet-14395648754114117926.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c -C [libcomet-14395648754114117926.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 -C [libcomet-14395648754114117926.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 -C [libcomet-14395648754114117926.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 -C [libcomet-14395648754114117926.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c -C [libcomet-14395648754114117926.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c -C [libcomet-14395648754114117926.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-14395648754114117926.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-14395648754114117926.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-14395648754114117926.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-14395648754114117926.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-14395648754114117926.dylib+0x106628] __rust_try+0x20 -C [libcomet-14395648754114117926.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-14395648754114117926.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-14395648754114117926.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-14395648754114117926.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3971+0x000000f8021c5bd8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3969+0x000000f8021c4da8.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2516+0x000000f801df1670.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3971+0x000000f8021c5bd8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3969+0x000000f8021c4da8.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2516+0x000000f801df1670.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 - -Registers: - x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000b64545270 - x4=0x000000017c5aad20 x5=0x0000000b661e1200 x6=0x0000000b7450a600 x7=0xfffff0003ffff800 - x8=0x000000f8021ade48 x9=0x0000000000000000 x10=0x000000f800000000 x11=0x0000000000000004 -x12=0x0000000139051af8 x13=0x0000000000000000 x14=0xfffffffffffdb1ac x15=0x0000000b66c48000 -x16=0x0000000000000040 x17=0x0000000b67100af0 x18=0x0000000000000000 x19=0x0000000b661e1200 -x20=0x000000017c5aad50 x21=0x000000017c5aad20 x22=0x0000000b6540ff18 x23=0x0000000713a9de98 -x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000b65c56f00 x27=0x0000000702bb6b20 -x28=0x0000000000000001 fp=0x000000017c5aad10 lr=0x000000010a7c152c sp=0x000000017c5aac00 -pc=0x000000010a7c1530 cpsr=0x0000000000001000 - -Register to memory mapping: - - x0=0x0 is NULL - x1=0x0000000000000007 is an unknown value - x2=0x0000000000000001 is an unknown value - x3=0x0000000b64545270 points into unknown readable memory: 0x0000000139051af8 | f8 1a 05 39 01 00 00 00 - x4=0x000000017c5aad20 is pointing into the stack for thread: 0x0000000b661e1200 - x5=0x0000000b661e1200 is a thread - x6=0x0000000b7450a600 points into unknown readable memory: 0x0000000b66578900 | 00 89 57 66 0b 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=0x000000f8021ade48 is pointing into metadata - x9=0x0 is NULL -x10=0x000000f800000000 is pointing into metadata -x11=0x0000000000000004 is an unknown value -x12={method} {0x0000000139051af8} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' -x13=0x0 is NULL -x14=0xfffffffffffdb1ac is an unknown value -x15=0x0000000b66c48000 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -x16=0x0000000000000040 is an unknown value -x17=0x0000000b67100af0 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff -x18=0x0 is NULL -x19=0x0000000b661e1200 is a thread -x20=0x000000017c5aad50 is pointing into the stack for thread: 0x0000000b661e1200 -x21=0x000000017c5aad20 is pointing into the stack for thread: 0x0000000b661e1200 -x22=0x0000000b6540ff18 points into unknown readable memory: 0x0000000713a9de98 | 98 de a9 13 07 00 00 00 -x23=0x0000000713a9de98 is an oop: org.apache.comet.CometHandleBatchIterator -{0x0000000713a9de98} - klass: 'org/apache/comet/CometHandleBatchIterator' - - ---- fields (total size 2 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x0000000713f31818} (e27e6303) -x24=0x0000000000000001 is an unknown value -x25=0x0 is NULL -x26=0x0000000b65c56f00 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x27=0x0000000702bb6b20 is an oop: [Z -{0x0000000702bb6b20} - klass: {type array bool} - - length: 9 -x28=0x0000000000000001 is an unknown value - - -Top of Stack: (sp=0x000000017c5aac00) -0x000000017c5aac00: 000000017c5a52c0 0000000000000001 -0x000000017c5aac10: 0000000b747601e0 0000000153212428 -0x000000017c5aac20: 0000000b71b3eac0 0000000b6c79d380 -0x000000017c5aac30: 000000017c5aac67 0000000000000000 -0x000000017c5aac40: 000000017c5aac80 00000001507800d0 -0x000000017c5aac50: 0000000000000081 0000000b664b5ed8 -0x000000017c5aac60: 0000000b74760320 0000000b664b5ed8 -0x000000017c5aac70: 0000000b664b5ed8 ff00ffffffffffff -0x000000017c5aac80: 000000017c5aad10 00000001507aff58 -0x000000017c5aac90: 0000000000000063 000000017c5aad20 -0x000000017c5aaca0: 000000017c5aad10 000000010a7ddf24 -0x000000017c5aacb0: 5bfd0d834fab00d0 0000000b661e1200 -0x000000017c5aacc0: 0000000b661e1200 0000000702bb6b20 -0x000000017c5aacd0: 0000000138fd9018 0000000000000000 -0x000000017c5aace0: 0000000b661e14b0 0000000000000008 -0x000000017c5aacf0: 0000000b668e81e0 0000000b64545270 -0x000000017c5aad00: 0000000000000001 0000000b661e1200 -0x000000017c5aad10: 000000017c5aadc0 000000010a7c560c -0x000000017c5aad20: 000000010af066b8 000000f80045a1d0 -0x000000017c5aad30: 000000010000000a 0000000000000014 -0x000000017c5aad40: 0000000000000000 0000000000000008 -0x000000017c5aad50: 000000000000000a 000000010af6ff90 -0x000000017c5aad60: 0000000b661e1200 0000000000000000 -0x000000017c5aad70: 000000017c5aadc0 0000000149e84c58 -0x000000017c5aad80: 0000000138fd9018 0000000702b3d680 -0x000000017c5aad90: 000000017c5b1818 0000000702b3d078 -0x000000017c5aada0: 0000000000000000 000000010af6ff90 -0x000000017c5aadb0: 000000017c5b1738 000000011a2ab72c -0x000000017c5aadc0: 000000017c5ac980 0000000149e49560 -0x000000017c5aadd0: 0000000000000000 0000000000000081 -0x000000017c5aade0: 0000000000000082 0000000b74760320 -0x000000017c5aadf0: 0000000000000082 ffff00ff00ff00ff - -Instructions: (pc=0x000000010a7c1530) -0x000000010a7c1430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 -0x000000010a7c1440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 -0x000000010a7c1450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 -0x000000010a7c1460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 -0x000000010a7c1470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 -0x000000010a7c1480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 -0x000000010a7c1490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 -0x000000010a7c14a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 -0x000000010a7c14b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa -0x000000010a7c14c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 -0x000000010a7c14d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 -0x000000010a7c14e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 -0x000000010a7c14f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a -0x000000010a7c1500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa -0x000000010a7c1510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 -0x000000010a7c1520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa -0x000000010a7c1530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 -0x000000010a7c1540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 -0x000000010a7c1550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 -0x000000010a7c1560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 -0x000000010a7c1570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 -0x000000010a7c1580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 -0x000000010a7c1590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa -0x000000010a7c15a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 -0x000000010a7c15b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 -0x000000010a7c15c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 -0x000000010a7c15d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 -0x000000010a7c15e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 -0x000000010a7c15f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa -0x000000010a7c1600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 -0x000000010a7c1610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 -0x000000010a7c1620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x000000017c5a52c0 is pointing into the stack for thread: 0x0000000b661e1200 -stack at sp + 1 slots: 0x0000000000000001 is an unknown value -stack at sp + 2 slots: 0x0000000b747601e0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -stack at sp + 3 slots: 0x0000000153212428: _ZN5comet7parquet15parquet_support15PARQUET_OPTIONS17h20af737816b13564E+0x28650 in /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib at 0x0000000149dd0000 -stack at sp + 4 slots: 0x0000000b71b3eac0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -stack at sp + 5 slots: 0x0000000b6c79d380 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -stack at sp + 6 slots: 0x000000017c5aac67 is pointing into the stack for thread: 0x0000000b661e1200 -stack at sp + 7 slots: 0x0 is NULL - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x0000000b63d63900, length=77, elements={ -0x0000000104f0c330, 0x0000000104f1a620, 0x0000000b668e4000, 0x0000000b668e4c00, -0x0000000b668e5200, 0x0000000b668e5800, 0x0000000b668e5e00, 0x0000000b668e6400, -0x0000000b668e6a00, 0x0000000b668e7000, 0x0000000b66932400, 0x0000000b66b74600, -0x0000000b66725e00, 0x0000000b65437000, 0x0000000b65434000, 0x0000000b66b0f000, -0x0000000b66b76a00, 0x0000000b66b75200, 0x0000000b65005e00, 0x0000000b65006400, -0x0000000b65006a00, 0x0000000b65007000, 0x0000000b65007600, 0x0000000b65014000, -0x0000000b65014600, 0x0000000b65014c00, 0x0000000b65017000, 0x0000000b65529200, -0x0000000b65529800, 0x0000000b6552aa00, 0x0000000b6552b000, 0x0000000b65094000, -0x0000000b65094600, 0x0000000b65094c00, 0x0000000b65095800, 0x0000000b65095e00, -0x0000000b65097000, 0x0000000b65097600, 0x0000000b650b4600, 0x0000000b650b5200, -0x0000000b650b5800, 0x0000000b650b5e00, 0x0000000b650b6400, 0x0000000b65136a00, -0x0000000b65178600, 0x0000000b65179200, 0x0000000b6517b600, 0x0000000b64fe7000, -0x0000000b661e1200, 0x0000000b661e1800, 0x0000000b661e1e00, 0x0000000b661e2400, -0x0000000b661e2a00, 0x0000000b6679b000, 0x0000000b65ca0000, 0x0000000b65ca0c00, -0x0000000b65ca1800, 0x0000000b6619b600, 0x0000000b65663600, 0x0000000b660ac000, -0x0000000b6517b000, 0x0000000b64460000, 0x0000000b64460600, 0x0000000b64460c00, -0x0000000b64461200, 0x0000000b64461800, 0x0000000b6451f600, 0x0000000b668e7600, -0x0000000b65684000, 0x0000000b66930000, 0x0000000b66b0e400, 0x0000000b66198000, -0x0000000b661e3600, 0x0000000b6451c600, 0x0000000b64463600, 0x0000000b63d64000, -0x0000000b63d64600 -} - -Java Threads: ( => current thread ) - 0x0000000104f0c330 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5635, stack(0x000000016b690000,0x000000016ba93000)] - 0x0000000104f1a620 JavaThread "Reference Handler" daemon [_thread_blocked, id=23299, stack(0x000000016c6e4000,0x000000016cae7000)] - 0x0000000b668e4000 JavaThread "Finalizer" daemon [_thread_blocked, id=31491, stack(0x000000016caf0000,0x000000016cef3000)] - 0x0000000b668e4c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=29699, stack(0x000000016d014000,0x000000016d417000)] - 0x0000000b668e5200 JavaThread "Service Thread" daemon [_thread_blocked, id=29187, stack(0x000000016d420000,0x000000016d823000)] - 0x0000000b668e5800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=28675, stack(0x000000016d82c000,0x000000016dc2f000)] - 0x0000000b668e5e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=25347, stack(0x000000016dc38000,0x000000016de3b000)] - 0x0000000b668e6400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27907, stack(0x000000016de44000,0x000000016e047000)] - 0x0000000b668e6a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27395, stack(0x000000016e050000,0x000000016e453000)] - 0x0000000b668e7000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=25603, stack(0x000000016e45c000,0x000000016e85f000)] - 0x0000000b66932400 JavaThread "Notification Thread" daemon [_thread_blocked, id=42499, stack(0x000000016f2a4000,0x000000016f6a7000)] - 0x0000000b66b74600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=45059, stack(0x00000001725c4000,0x00000001729c7000)] - 0x0000000b66725e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=62211, stack(0x000000017340c000,0x000000017380f000)] - 0x0000000b65437000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=46851, stack(0x0000000173818000,0x0000000173c1b000)] - 0x0000000b65434000 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=47107, stack(0x0000000173c24000,0x0000000174027000)] - 0x0000000b66b0f000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=61443, stack(0x0000000174030000,0x0000000174433000)] - 0x0000000b66b76a00 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=47619, stack(0x000000017443c000,0x000000017483f000)] - 0x0000000b66b75200 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=60931, stack(0x0000000174848000,0x0000000174c4b000)] - 0x0000000b65005e00 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=60675, stack(0x0000000174c54000,0x0000000175057000)] - 0x0000000b65006400 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=60419, stack(0x0000000175060000,0x0000000175463000)] - 0x0000000b65006a00 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=60163, stack(0x000000017546c000,0x000000017586f000)] - 0x0000000b65007000 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=48643, stack(0x0000000175878000,0x0000000175c7b000)] - 0x0000000b65007600 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=49155, stack(0x0000000175c84000,0x0000000176087000)] - 0x0000000b65014000 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=49667, stack(0x0000000176090000,0x0000000176493000)] - 0x0000000b65014600 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=49923, stack(0x000000017649c000,0x000000017689f000)] - 0x0000000b65014c00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=50179, stack(0x00000001768a8000,0x0000000176cab000)] - 0x0000000b65017000 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=59139, stack(0x0000000176cb4000,0x00000001770b7000)] - 0x0000000b65529200 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58883, stack(0x00000001770c0000,0x00000001774c3000)] - 0x0000000b65529800 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=51203, stack(0x00000001774cc000,0x00000001778cf000)] - 0x0000000b6552aa00 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=58627, stack(0x00000001778d8000,0x0000000177cdb000)] - 0x0000000b6552b000 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=58371, stack(0x0000000177ce4000,0x00000001780e7000)] - 0x0000000b65094000 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=51971, stack(0x00000001780f0000,0x00000001784f3000)] - 0x0000000b65094600 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=57859, stack(0x00000001784fc000,0x00000001788ff000)] - 0x0000000b65094c00 JavaThread "task-abort-timer" daemon [_thread_blocked, id=52483, stack(0x0000000178908000,0x0000000178d0b000)] - 0x0000000b65095800 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=57091, stack(0x0000000178d14000,0x0000000179117000)] - 0x0000000b65095e00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52739, stack(0x0000000179120000,0x0000000179523000)] - 0x0000000b65097000 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=56323, stack(0x000000017952c000,0x000000017992f000)] - 0x0000000b65097600 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=53251, stack(0x0000000179938000,0x0000000179d3b000)] - 0x0000000b650b4600 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=53763, stack(0x0000000179d44000,0x000000017a147000)] - 0x0000000b650b5200 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54019, stack(0x000000017a150000,0x000000017a553000)] - 0x0000000b650b5800 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54531, stack(0x000000017a55c000,0x000000017a95f000)] - 0x0000000b650b5e00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=54787, stack(0x000000017a968000,0x000000017ad6b000)] - 0x0000000b650b6400 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55299, stack(0x000000017ad74000,0x000000017b177000)] - 0x0000000b65136a00 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=65539, stack(0x000000017b180000,0x000000017b583000)] - 0x0000000b65178600 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=86787, stack(0x000000017b58c000,0x000000017b98f000)] - 0x0000000b65179200 JavaThread "Timer-0" [_thread_blocked, id=86275, stack(0x000000017b998000,0x000000017bd9b000)] - 0x0000000b6517b600 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=85763, stack(0x000000017bda4000,0x000000017c1a7000)] - 0x0000000b64fe7000 JavaThread "process reaper" daemon [_thread_blocked, id=34575, stack(0x000000016fcd4000,0x000000016fd0b000)] -=>0x0000000b661e1200 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=63499, stack(0x000000017c1b0000,0x000000017c5b3000)] - 0x0000000b661e1800 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=41739, stack(0x000000017c5bc000,0x000000017c9bf000)] - 0x0000000b661e1e00 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=24339, stack(0x000000017c9c8000,0x000000017cdcb000)] - 0x0000000b661e2400 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=65795, stack(0x000000017cdd4000,0x000000017d1d7000)] - 0x0000000b661e2a00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=66307, stack(0x000000017d1e0000,0x000000017d5e3000)] - 0x0000000b6679b000 JavaThread "process reaper" daemon [_thread_blocked, id=84483, stack(0x000000016cf88000,0x000000016cfbf000)] - 0x0000000b65ca0000 JavaThread "process reaper" daemon [_thread_blocked, id=67331, stack(0x000000016cfc8000,0x000000016cfff000)] - 0x0000000b65ca0c00 JavaThread "process reaper" daemon [_thread_blocked, id=68099, stack(0x000000016fd14000,0x000000016fd4b000)] - 0x0000000b65ca1800 JavaThread "process reaper" daemon [_thread_blocked, id=82947, stack(0x000000016fd54000,0x000000016fd8b000)] - 0x0000000b6619b600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=34063, stack(0x000000017d5ec000,0x000000017d9ef000)] - 0x0000000b65663600 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=68623, stack(0x000000017d9f8000,0x000000017ddfb000)] - 0x0000000b660ac000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=83507, stack(0x000000017de04000,0x000000017e207000)] - 0x0000000b6517b000 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=83219, stack(0x000000017e210000,0x000000017e613000)] - 0x0000000b64460000 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69179, stack(0x000000017e61c000,0x000000017ea1f000)] - 0x0000000b64460600 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=82747, stack(0x000000017ea28000,0x000000017ee2b000)] - 0x0000000b64460c00 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=82179, stack(0x000000017ee34000,0x000000017f237000)] - 0x0000000b64461200 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=69891, stack(0x000000017f240000,0x000000017f643000)] - 0x0000000b64461800 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=81923, stack(0x000000017f64c000,0x000000017fa4f000)] - 0x0000000b6451f600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=79879, stack(0x000000016e868000,0x000000016ea6b000)] - 0x0000000b668e7600 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=72207, stack(0x000000016ea74000,0x000000016ec77000)] - 0x0000000b65684000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=71947, stack(0x000000016ec80000,0x000000016ee83000)] - 0x0000000b66930000 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=25871, stack(0x000000016ee8c000,0x000000016f08f000)] - 0x0000000b66b0e400 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=43331, stack(0x000000016f098000,0x000000016f29b000)] - 0x0000000b66198000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=33311, stack(0x000000016f8bc000,0x000000016fcbf000)] - 0x0000000b661e3600 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=42787, stack(0x000000030041c000,0x000000030081f000)] - 0x0000000b6451c600 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=26635, stack(0x0000000300828000,0x0000000300c2b000)] - 0x0000000b64463600 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=26379, stack(0x0000000300e58000,0x000000030125b000)] - 0x0000000b63d64000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=84747, stack(0x0000000301264000,0x0000000301667000)] - 0x0000000b63d64600 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=79363, stack(0x0000000301670000,0x0000000301a73000)] - -Other Threads: - 0x0000000104f199a0 VMThread "VM Thread" [stack: 0x000000016c4d8000,0x000000016c6db000] [id=19715] - 0x0000000b66980600 WatcherThread [stack: 0x000000016f6b0000,0x000000016f8b3000] [id=33539] - 0x0000000104f13370 GCTaskThread "GC Thread#0" [stack: 0x000000016ba9c000,0x000000016bc9f000] [id=14083] - 0x0000000b66524f00 GCTaskThread "GC Thread#1" [stack: 0x000000016fee0000,0x00000001700e3000] [id=35075] - 0x0000000b66525200 GCTaskThread "GC Thread#2" [stack: 0x00000001700ec000,0x00000001702ef000] [id=35587] - 0x0000000b66525500 GCTaskThread "GC Thread#3" [stack: 0x00000001702f8000,0x00000001704fb000] [id=41219] - 0x0000000b66525800 GCTaskThread "GC Thread#4" [stack: 0x0000000170504000,0x0000000170707000] [id=40707] - 0x0000000b66525b00 GCTaskThread "GC Thread#5" [stack: 0x0000000170710000,0x0000000170913000] [id=36099] - 0x0000000b66525e00 GCTaskThread "GC Thread#6" [stack: 0x000000017091c000,0x0000000170b1f000] [id=40451] - 0x0000000b66526100 GCTaskThread "GC Thread#7" [stack: 0x0000000170b28000,0x0000000170d2b000] [id=40195] - 0x0000000b66526400 GCTaskThread "GC Thread#8" [stack: 0x0000000170d34000,0x0000000170f37000] [id=39683] - 0x0000000b66526700 GCTaskThread "GC Thread#9" [stack: 0x0000000170f40000,0x0000000171143000] [id=37123] - 0x0000000b66526a00 GCTaskThread "GC Thread#10" [stack: 0x000000017114c000,0x000000017134f000] [id=39427] - 0x0000000b66526d00 GCTaskThread "GC Thread#11" [stack: 0x0000000171358000,0x000000017155b000] [id=37635] - 0x0000000b66527000 GCTaskThread "GC Thread#12" [stack: 0x0000000171564000,0x0000000171767000] [id=37891] - 0x0000000b66527300 GCTaskThread "GC Thread#13" [stack: 0x0000000171770000,0x0000000171973000] [id=38403] - 0x0000000b66527600 GCTaskThread "GC Thread#14" [stack: 0x000000017197c000,0x0000000171b7f000] [id=43523] - 0x0000000b66527900 GCTaskThread "GC Thread#15" [stack: 0x0000000171b88000,0x0000000171d8b000] [id=65283] - 0x0000000b66527c00 GCTaskThread "GC Thread#16" [stack: 0x0000000171d94000,0x0000000171f97000] [id=44035] - 0x0000000b66b08000 GCTaskThread "GC Thread#17" [stack: 0x0000000171fa0000,0x00000001721a3000] [id=64771] - 0x0000000b66b08300 GCTaskThread "GC Thread#18" [stack: 0x00000001721ac000,0x00000001723af000] [id=64259] - 0x0000000b66b08600 GCTaskThread "GC Thread#19" [stack: 0x00000001723b8000,0x00000001725bb000] [id=44803] - 0x0000000104f13f70 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016bca8000,0x000000016beab000] [id=13827] - 0x0000000104f146e0 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016beb4000,0x000000016c0b7000] [id=13315] - 0x0000000b65430f00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000172bdc000,0x0000000172ddf000] [id=63235] - 0x0000000b65431500 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000172de8000,0x0000000172feb000] [id=45571] - 0x0000000b65431800 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000172ff4000,0x00000001731f7000] [id=46083] - 0x0000000b65431b00 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000173200000,0x0000000173403000] [id=62467] - 0x0000000104f16590 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016c0c0000,0x000000016c2c3000] [id=21507] - 0x0000000104f16c90 ConcurrentGCThread "G1 Service" [stack: 0x000000016c2cc000,0x000000016c4cf000] [id=16643] - -Threads with active compile tasks: - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x0000000700000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x000000f800000000-0x000000f800bc0000-0x000000f800bc0000), size 12320768, SharedBaseAddress: 0x000000f800000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x000000f801000000-0x000000f841000000, reserved size: 1073741824 -Narrow klass base: 0x000000f800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 348160K, used 181662K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 15 young (30720K), 4 survivors (8192K) - Metaspace used 151503K, committed 152768K, reserved 1245184K - class space used 18223K, committed 18816K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000700000000, 0x0000000700200000, 0x0000000700200000|100%|HS| |TAMS 0x0000000700200000, 0x0000000700200000| Complete -| 1|0x0000000700200000, 0x0000000700400000, 0x0000000700400000|100%|HS| |TAMS 0x0000000700400000, 0x0000000700400000| Complete -| 2|0x0000000700400000, 0x0000000700600000, 0x0000000700600000|100%|HC| |TAMS 0x0000000700600000, 0x0000000700600000| Complete -| 3|0x0000000700600000, 0x0000000700800000, 0x0000000700800000|100%|HS| |TAMS 0x0000000700800000, 0x0000000700800000| Complete -| 4|0x0000000700800000, 0x0000000700a00000, 0x0000000700a00000|100%|HC| |TAMS 0x0000000700a00000, 0x0000000700a00000| Complete -| 5|0x0000000700a00000, 0x0000000700c00000, 0x0000000700c00000|100%|HC| |TAMS 0x0000000700c00000, 0x0000000700c00000| Complete -| 6|0x0000000700c00000, 0x0000000700e00000, 0x0000000700e00000|100%|HC| |TAMS 0x0000000700e00000, 0x0000000700e00000| Complete -| 7|0x0000000700e00000, 0x0000000701000000, 0x0000000701000000|100%|HC| |TAMS 0x0000000701000000, 0x0000000701000000| Complete -| 8|0x0000000701000000, 0x0000000701200000, 0x0000000701200000|100%|HC| |TAMS 0x0000000701200000, 0x0000000701200000| Complete -| 9|0x0000000701200000, 0x0000000701400000, 0x0000000701400000|100%|HC| |TAMS 0x0000000701400000, 0x0000000701400000| Complete -| 10|0x0000000701400000, 0x0000000701600000, 0x0000000701600000|100%|HC| |TAMS 0x0000000701600000, 0x0000000701600000| Complete -| 11|0x0000000701600000, 0x0000000701800000, 0x0000000701800000|100%|HS| |TAMS 0x0000000701800000, 0x0000000701800000| Complete -| 12|0x0000000701800000, 0x0000000701a00000, 0x0000000701a00000|100%|HS| |TAMS 0x0000000701a00000, 0x0000000701a00000| Complete -| 13|0x0000000701a00000, 0x0000000701c00000, 0x0000000701c00000|100%|HC| |TAMS 0x0000000701c00000, 0x0000000701c00000| Complete -| 14|0x0000000701c00000, 0x0000000701e00000, 0x0000000701e00000|100%|HS| |TAMS 0x0000000701e00000, 0x0000000701e00000| Complete -| 15|0x0000000701e00000, 0x0000000702000000, 0x0000000702000000|100%|HC| |TAMS 0x0000000702000000, 0x0000000702000000| Complete -| 16|0x0000000702000000, 0x0000000702200000, 0x0000000702200000|100%|HS| |TAMS 0x0000000702200000, 0x0000000702200000| Complete -| 17|0x0000000702200000, 0x0000000702400000, 0x0000000702400000|100%|HS| |TAMS 0x0000000702400000, 0x0000000702400000| Complete -| 18|0x0000000702400000, 0x0000000702600000, 0x0000000702600000|100%|HC| |TAMS 0x0000000702600000, 0x0000000702600000| Complete -| 19|0x0000000702600000, 0x0000000702800000, 0x0000000702800000|100%| O| |TAMS 0x0000000702800000, 0x0000000702800000| Untracked -| 20|0x0000000702800000, 0x0000000702a00000, 0x0000000702a00000|100%| O| |TAMS 0x0000000702a00000, 0x0000000702a00000| Untracked -| 21|0x0000000702a00000, 0x0000000702c00000, 0x0000000702c00000|100%| O| |TAMS 0x0000000702c00000, 0x0000000702c00000| Untracked -| 22|0x0000000702c00000, 0x0000000702e00000, 0x0000000702e00000|100%| O| |TAMS 0x0000000702e00000, 0x0000000702e00000| Untracked -| 23|0x0000000702e00000, 0x0000000703000000, 0x0000000703000000|100%| O| |TAMS 0x0000000703000000, 0x0000000703000000| Untracked -| 24|0x0000000703000000, 0x0000000703200000, 0x0000000703200000|100%| O| |TAMS 0x0000000703200000, 0x0000000703200000| Untracked -| 25|0x0000000703200000, 0x0000000703400000, 0x0000000703400000|100%| O| |TAMS 0x0000000703400000, 0x0000000703400000| Untracked -| 26|0x0000000703400000, 0x0000000703565000, 0x0000000703600000| 69%| O| |TAMS 0x0000000703565000, 0x0000000703565000| Untracked -| 27|0x0000000703600000, 0x0000000703726e00, 0x0000000703800000| 57%| O| |TAMS 0x0000000703726e00, 0x0000000703726e00| Untracked -| 28|0x0000000703800000, 0x0000000703a00000, 0x0000000703a00000|100%| O| |TAMS 0x0000000703a00000, 0x0000000703a00000| Untracked -| 29|0x0000000703a00000, 0x0000000703c00000, 0x0000000703c00000|100%| O| |TAMS 0x0000000703c00000, 0x0000000703c00000| Untracked -| 30|0x0000000703c00000, 0x0000000703e00000, 0x0000000703e00000|100%| O| |TAMS 0x0000000703e00000, 0x0000000703e00000| Untracked -| 31|0x0000000703e00000, 0x0000000704000000, 0x0000000704000000|100%| O| |TAMS 0x0000000704000000, 0x0000000704000000| Untracked -| 32|0x0000000704000000, 0x0000000704200000, 0x0000000704200000|100%| O| |TAMS 0x0000000704200000, 0x0000000704200000| Untracked -| 33|0x0000000704200000, 0x0000000704400000, 0x0000000704400000|100%| O| |TAMS 0x0000000704400000, 0x0000000704400000| Untracked -| 34|0x0000000704400000, 0x0000000704600000, 0x0000000704600000|100%| O| |TAMS 0x0000000704600000, 0x0000000704600000| Untracked -| 35|0x0000000704600000, 0x0000000704800000, 0x0000000704800000|100%| O| |TAMS 0x0000000704800000, 0x0000000704800000| Untracked -| 36|0x0000000704800000, 0x0000000704a00000, 0x0000000704a00000|100%| O| |TAMS 0x0000000704a00000, 0x0000000704a00000| Untracked -| 37|0x0000000704a00000, 0x0000000704c00000, 0x0000000704c00000|100%| O| |TAMS 0x0000000704c00000, 0x0000000704c00000| Untracked -| 38|0x0000000704c00000, 0x0000000704e00000, 0x0000000704e00000|100%| O| |TAMS 0x0000000704e00000, 0x0000000704e00000| Untracked -| 39|0x0000000704e00000, 0x0000000705000000, 0x0000000705000000|100%| O| |TAMS 0x0000000705000000, 0x0000000705000000| Untracked -| 40|0x0000000705000000, 0x0000000705200000, 0x0000000705200000|100%| O| |TAMS 0x0000000705200000, 0x0000000705200000| Untracked -| 41|0x0000000705200000, 0x0000000705400000, 0x0000000705400000|100%| O| |TAMS 0x0000000705400000, 0x0000000705400000| Untracked -| 42|0x0000000705400000, 0x00000007055d3c00, 0x0000000705600000| 91%| O| |TAMS 0x00000007055d3c00, 0x00000007055d3c00| Untracked -| 43|0x0000000705600000, 0x0000000705800000, 0x0000000705800000|100%|HS| |TAMS 0x0000000705600000, 0x0000000705800000| Complete -| 44|0x0000000705800000, 0x0000000705a00000, 0x0000000705a00000|100%|HS| |TAMS 0x0000000705800000, 0x0000000705a00000| Complete -| 45|0x0000000705a00000, 0x0000000705c00000, 0x0000000705c00000|100%|HS| |TAMS 0x0000000705a00000, 0x0000000705c00000| Complete -| 46|0x0000000705c00000, 0x0000000705e00000, 0x0000000705e00000|100%|HS| |TAMS 0x0000000705c00000, 0x0000000705e00000| Complete -| 47|0x0000000705e00000, 0x0000000706000000, 0x0000000706000000|100%|HS| |TAMS 0x0000000705e00000, 0x0000000706000000| Complete -| 48|0x0000000706000000, 0x0000000706200000, 0x0000000706200000|100%|HS| |TAMS 0x0000000706000000, 0x0000000706200000| Complete -| 49|0x0000000706200000, 0x0000000706400000, 0x0000000706400000|100%|HS| |TAMS 0x0000000706200000, 0x0000000706400000| Complete -| 50|0x0000000706400000, 0x0000000706600000, 0x0000000706600000|100%|HS| |TAMS 0x0000000706400000, 0x0000000706600000| Complete -| 51|0x0000000706600000, 0x0000000706800000, 0x0000000706800000|100%|HS| |TAMS 0x0000000706600000, 0x0000000706800000| Complete -| 52|0x0000000706800000, 0x0000000706a00000, 0x0000000706a00000|100%|HS| |TAMS 0x0000000706800000, 0x0000000706a00000| Complete -| 53|0x0000000706a00000, 0x0000000706c00000, 0x0000000706c00000|100%| O| |TAMS 0x0000000706a00000, 0x0000000706c00000| Untracked -| 54|0x0000000706c00000, 0x0000000706e00000, 0x0000000706e00000|100%| O| |TAMS 0x0000000706c00000, 0x0000000706e00000| Untracked -| 55|0x0000000706e00000, 0x0000000707000000, 0x0000000707000000|100%| O| |TAMS 0x0000000706e00000, 0x0000000707000000| Untracked -| 56|0x0000000707000000, 0x0000000707200000, 0x0000000707200000|100%| O| |TAMS 0x0000000707000000, 0x0000000707200000| Untracked -| 57|0x0000000707200000, 0x0000000707400000, 0x0000000707400000|100%| O| |TAMS 0x0000000707200000, 0x0000000707400000| Untracked -| 58|0x0000000707400000, 0x0000000707600000, 0x0000000707600000|100%| O| |TAMS 0x0000000707400000, 0x0000000707600000| Untracked -| 59|0x0000000707600000, 0x0000000707800000, 0x0000000707800000|100%| O| |TAMS 0x0000000707600000, 0x0000000707800000| Untracked -| 60|0x0000000707800000, 0x000000070797ba00, 0x0000000707a00000| 74%| O| |TAMS 0x0000000707800000, 0x000000070797ba00| Untracked -| 61|0x0000000707a00000, 0x0000000707a00000, 0x0000000707c00000| 0%| F| |TAMS 0x0000000707a00000, 0x0000000707a00000| Untracked -| 62|0x0000000707c00000, 0x0000000707c00000, 0x0000000707e00000| 0%| F| |TAMS 0x0000000707c00000, 0x0000000707c00000| Untracked -| 63|0x0000000707e00000, 0x0000000708000000, 0x0000000708000000|100%| O| |TAMS 0x0000000707e00000, 0x0000000708000000| Untracked -| 64|0x0000000708000000, 0x0000000708200000, 0x0000000708200000|100%| O| |TAMS 0x0000000708000000, 0x0000000708200000| Untracked -| 65|0x0000000708200000, 0x0000000708400000, 0x0000000708400000|100%| O| |TAMS 0x0000000708200000, 0x0000000708400000| Untracked -| 66|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%|HS| |TAMS 0x0000000708400000, 0x0000000708400000| Complete -| 67|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%|HC| |TAMS 0x0000000708600000, 0x0000000708600000| Complete -| 68|0x0000000708800000, 0x0000000708a00000, 0x0000000708a00000|100%|HC| |TAMS 0x0000000708800000, 0x0000000708800000| Complete -| 69|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked -| 70|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked -| 71|0x0000000708e00000, 0x0000000709000000, 0x0000000709000000|100%| O| |TAMS 0x0000000708e00000, 0x0000000709000000| Untracked -| 72|0x0000000709000000, 0x0000000709200000, 0x0000000709200000|100%| O| |TAMS 0x0000000709000000, 0x0000000709200000| Untracked -| 73|0x0000000709200000, 0x0000000709400000, 0x0000000709400000|100%| O| |TAMS 0x0000000709200000, 0x0000000709400000| Untracked -| 74|0x0000000709400000, 0x0000000709600000, 0x0000000709600000|100%| O| |TAMS 0x0000000709400000, 0x0000000709600000| Untracked -| 75|0x0000000709600000, 0x0000000709800000, 0x0000000709800000|100%| O| |TAMS 0x0000000709600000, 0x0000000709800000| Untracked -| 76|0x0000000709800000, 0x0000000709a00000, 0x0000000709a00000|100%| O| |TAMS 0x0000000709800000, 0x0000000709a00000| Untracked -| 77|0x0000000709a00000, 0x0000000709c00000, 0x0000000709c00000|100%| O| |TAMS 0x0000000709a00000, 0x0000000709c00000| Untracked -| 78|0x0000000709c00000, 0x0000000709e00000, 0x0000000709e00000|100%| O| |TAMS 0x0000000709c00000, 0x0000000709e00000| Untracked -| 79|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked -| 80|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked -| 81|0x000000070a200000, 0x000000070a200000, 0x000000070a400000| 0%| F| |TAMS 0x000000070a200000, 0x000000070a200000| Untracked -| 82|0x000000070a400000, 0x000000070a400000, 0x000000070a600000| 0%| F| |TAMS 0x000000070a400000, 0x000000070a400000| Untracked -| 83|0x000000070a600000, 0x000000070a600000, 0x000000070a800000| 0%| F| |TAMS 0x000000070a600000, 0x000000070a600000| Untracked -| 84|0x000000070a800000, 0x000000070a800000, 0x000000070aa00000| 0%| F| |TAMS 0x000000070a800000, 0x000000070a800000| Untracked -| 85|0x000000070aa00000, 0x000000070aa00000, 0x000000070ac00000| 0%| F| |TAMS 0x000000070aa00000, 0x000000070aa00000| Untracked -| 86|0x000000070ac00000, 0x000000070ac00000, 0x000000070ae00000| 0%| F| |TAMS 0x000000070ac00000, 0x000000070ac00000| Untracked -| 87|0x000000070ae00000, 0x000000070ae00000, 0x000000070b000000| 0%| F| |TAMS 0x000000070ae00000, 0x000000070ae00000| Untracked -| 88|0x000000070b000000, 0x000000070b000000, 0x000000070b200000| 0%| F| |TAMS 0x000000070b000000, 0x000000070b000000| Untracked -| 89|0x000000070b200000, 0x000000070b200000, 0x000000070b400000| 0%| F| |TAMS 0x000000070b200000, 0x000000070b200000| Untracked -| 90|0x000000070b400000, 0x000000070b400000, 0x000000070b600000| 0%| F| |TAMS 0x000000070b400000, 0x000000070b400000| Untracked -| 91|0x000000070b600000, 0x000000070b600000, 0x000000070b800000| 0%| F| |TAMS 0x000000070b600000, 0x000000070b600000| Untracked -| 92|0x000000070b800000, 0x000000070b800000, 0x000000070ba00000| 0%| F| |TAMS 0x000000070b800000, 0x000000070b800000| Untracked -| 93|0x000000070ba00000, 0x000000070ba00000, 0x000000070bc00000| 0%| F| |TAMS 0x000000070ba00000, 0x000000070ba00000| Untracked -| 94|0x000000070bc00000, 0x000000070bc00000, 0x000000070be00000| 0%| F| |TAMS 0x000000070bc00000, 0x000000070bc00000| Untracked -| 95|0x000000070be00000, 0x000000070be00000, 0x000000070c000000| 0%| F| |TAMS 0x000000070be00000, 0x000000070be00000| Untracked -| 96|0x000000070c000000, 0x000000070c000000, 0x000000070c200000| 0%| F| |TAMS 0x000000070c000000, 0x000000070c000000| Untracked -| 97|0x000000070c200000, 0x000000070c200000, 0x000000070c400000| 0%| F| |TAMS 0x000000070c200000, 0x000000070c200000| Untracked -| 98|0x000000070c400000, 0x000000070c400000, 0x000000070c600000| 0%| F| |TAMS 0x000000070c400000, 0x000000070c400000| Untracked -| 99|0x000000070c600000, 0x000000070c600000, 0x000000070c800000| 0%| F| |TAMS 0x000000070c600000, 0x000000070c600000| Untracked -| 100|0x000000070c800000, 0x000000070c800000, 0x000000070ca00000| 0%| F| |TAMS 0x000000070c800000, 0x000000070c800000| Untracked -| 101|0x000000070ca00000, 0x000000070ca00000, 0x000000070cc00000| 0%| F| |TAMS 0x000000070ca00000, 0x000000070ca00000| Untracked -| 102|0x000000070cc00000, 0x000000070cc00000, 0x000000070ce00000| 0%| F| |TAMS 0x000000070cc00000, 0x000000070cc00000| Untracked -| 103|0x000000070ce00000, 0x000000070ce00000, 0x000000070d000000| 0%| F| |TAMS 0x000000070ce00000, 0x000000070ce00000| Untracked -| 104|0x000000070d000000, 0x000000070d000000, 0x000000070d200000| 0%| F| |TAMS 0x000000070d000000, 0x000000070d000000| Untracked -| 105|0x000000070d200000, 0x000000070d200000, 0x000000070d400000| 0%| F| |TAMS 0x000000070d200000, 0x000000070d200000| Untracked -| 106|0x000000070d400000, 0x000000070d400000, 0x000000070d600000| 0%| F| |TAMS 0x000000070d400000, 0x000000070d400000| Untracked -| 107|0x000000070d600000, 0x000000070d600000, 0x000000070d800000| 0%| F| |TAMS 0x000000070d600000, 0x000000070d600000| Untracked -| 108|0x000000070d800000, 0x000000070d800000, 0x000000070da00000| 0%| F| |TAMS 0x000000070d800000, 0x000000070d800000| Untracked -| 109|0x000000070da00000, 0x000000070da00000, 0x000000070dc00000| 0%| F| |TAMS 0x000000070da00000, 0x000000070da00000| Untracked -| 110|0x000000070dc00000, 0x000000070dc00000, 0x000000070de00000| 0%| F| |TAMS 0x000000070dc00000, 0x000000070dc00000| Untracked -| 111|0x000000070de00000, 0x000000070de00000, 0x000000070e000000| 0%| F| |TAMS 0x000000070de00000, 0x000000070de00000| Untracked -| 112|0x000000070e000000, 0x000000070e000000, 0x000000070e200000| 0%| F| |TAMS 0x000000070e000000, 0x000000070e000000| Untracked -| 113|0x000000070e200000, 0x000000070e200000, 0x000000070e400000| 0%| F| |TAMS 0x000000070e200000, 0x000000070e200000| Untracked -| 114|0x000000070e400000, 0x000000070e400000, 0x000000070e600000| 0%| F| |TAMS 0x000000070e400000, 0x000000070e400000| Untracked -| 115|0x000000070e600000, 0x000000070e600000, 0x000000070e800000| 0%| F| |TAMS 0x000000070e600000, 0x000000070e600000| Untracked -| 116|0x000000070e800000, 0x000000070e800000, 0x000000070ea00000| 0%| F| |TAMS 0x000000070e800000, 0x000000070e800000| Untracked -| 117|0x000000070ea00000, 0x000000070ea00000, 0x000000070ec00000| 0%| F| |TAMS 0x000000070ea00000, 0x000000070ea00000| Untracked -| 118|0x000000070ec00000, 0x000000070ec00000, 0x000000070ee00000| 0%| F| |TAMS 0x000000070ec00000, 0x000000070ec00000| Untracked -| 119|0x000000070ee00000, 0x000000070ee00000, 0x000000070f000000| 0%| F| |TAMS 0x000000070ee00000, 0x000000070ee00000| Untracked -| 120|0x000000070f000000, 0x000000070f000000, 0x000000070f200000| 0%| F| |TAMS 0x000000070f000000, 0x000000070f000000| Untracked -| 121|0x000000070f200000, 0x000000070f200000, 0x000000070f400000| 0%| F| |TAMS 0x000000070f200000, 0x000000070f200000| Untracked -| 122|0x000000070f400000, 0x000000070f400000, 0x000000070f600000| 0%| F| |TAMS 0x000000070f400000, 0x000000070f400000| Untracked -| 123|0x000000070f600000, 0x000000070f600000, 0x000000070f800000| 0%| F| |TAMS 0x000000070f600000, 0x000000070f600000| Untracked -| 124|0x000000070f800000, 0x000000070f800000, 0x000000070fa00000| 0%| F| |TAMS 0x000000070f800000, 0x000000070f800000| Untracked -| 125|0x000000070fa00000, 0x000000070fa00000, 0x000000070fc00000| 0%| F| |TAMS 0x000000070fa00000, 0x000000070fa00000| Untracked -| 126|0x000000070fc00000, 0x000000070fc00000, 0x000000070fe00000| 0%| F| |TAMS 0x000000070fc00000, 0x000000070fc00000| Untracked -| 127|0x000000070fe00000, 0x000000070fe00000, 0x0000000710000000| 0%| F| |TAMS 0x000000070fe00000, 0x000000070fe00000| Untracked -| 128|0x0000000710000000, 0x0000000710000000, 0x0000000710200000| 0%| F| |TAMS 0x0000000710000000, 0x0000000710000000| Untracked -| 129|0x0000000710200000, 0x0000000710200000, 0x0000000710400000| 0%| F| |TAMS 0x0000000710200000, 0x0000000710200000| Untracked -| 130|0x0000000710400000, 0x0000000710400000, 0x0000000710600000| 0%| F| |TAMS 0x0000000710400000, 0x0000000710400000| Untracked -| 131|0x0000000710600000, 0x0000000710600000, 0x0000000710800000| 0%| F| |TAMS 0x0000000710600000, 0x0000000710600000| Untracked -| 132|0x0000000710800000, 0x0000000710800000, 0x0000000710a00000| 0%| F| |TAMS 0x0000000710800000, 0x0000000710800000| Untracked -| 133|0x0000000710a00000, 0x0000000710a00000, 0x0000000710c00000| 0%| F| |TAMS 0x0000000710a00000, 0x0000000710a00000| Untracked -| 134|0x0000000710c00000, 0x0000000710c00000, 0x0000000710e00000| 0%| F| |TAMS 0x0000000710c00000, 0x0000000710c00000| Untracked -| 135|0x0000000710e00000, 0x0000000710e00000, 0x0000000711000000| 0%| F| |TAMS 0x0000000710e00000, 0x0000000710e00000| Untracked -| 136|0x0000000711000000, 0x0000000711000000, 0x0000000711200000| 0%| F| |TAMS 0x0000000711000000, 0x0000000711000000| Untracked -| 137|0x0000000711200000, 0x0000000711200000, 0x0000000711400000| 0%| F| |TAMS 0x0000000711200000, 0x0000000711200000| Untracked -| 138|0x0000000711400000, 0x0000000711400000, 0x0000000711600000| 0%| F| |TAMS 0x0000000711400000, 0x0000000711400000| Untracked -| 139|0x0000000711600000, 0x0000000711600000, 0x0000000711800000| 0%| F| |TAMS 0x0000000711600000, 0x0000000711600000| Untracked -| 140|0x0000000711800000, 0x0000000711800000, 0x0000000711a00000| 0%| F| |TAMS 0x0000000711800000, 0x0000000711800000| Untracked -| 141|0x0000000711a00000, 0x0000000711a00000, 0x0000000711c00000| 0%| F| |TAMS 0x0000000711a00000, 0x0000000711a00000| Untracked -| 142|0x0000000711c00000, 0x0000000711c00000, 0x0000000711e00000| 0%| F| |TAMS 0x0000000711c00000, 0x0000000711c00000| Untracked -| 143|0x0000000711e00000, 0x0000000711e00000, 0x0000000712000000| 0%| F| |TAMS 0x0000000711e00000, 0x0000000711e00000| Untracked -| 144|0x0000000712000000, 0x0000000712000000, 0x0000000712200000| 0%| F| |TAMS 0x0000000712000000, 0x0000000712000000| Untracked -| 145|0x0000000712200000, 0x0000000712200000, 0x0000000712400000| 0%| F| |TAMS 0x0000000712200000, 0x0000000712200000| Untracked -| 146|0x0000000712400000, 0x0000000712400000, 0x0000000712600000| 0%| F| |TAMS 0x0000000712400000, 0x0000000712400000| Untracked -| 147|0x0000000712600000, 0x0000000712600000, 0x0000000712800000| 0%| F| |TAMS 0x0000000712600000, 0x0000000712600000| Untracked -| 148|0x0000000712800000, 0x0000000712800000, 0x0000000712a00000| 0%| F| |TAMS 0x0000000712800000, 0x0000000712800000| Untracked -| 149|0x0000000712a00000, 0x0000000712a00000, 0x0000000712c00000| 0%| F| |TAMS 0x0000000712a00000, 0x0000000712a00000| Untracked -| 150|0x0000000712c00000, 0x0000000712c00000, 0x0000000712e00000| 0%| F| |TAMS 0x0000000712c00000, 0x0000000712c00000| Untracked -| 151|0x0000000712e00000, 0x0000000712e00000, 0x0000000713000000| 0%| F| |TAMS 0x0000000712e00000, 0x0000000712e00000| Untracked -| 152|0x0000000713000000, 0x0000000713000000, 0x0000000713200000| 0%| F| |TAMS 0x0000000713000000, 0x0000000713000000| Untracked -| 153|0x0000000713200000, 0x00000007133a4680, 0x0000000713400000| 82%| S|CS|TAMS 0x0000000713200000, 0x0000000713200000| Complete -| 154|0x0000000713400000, 0x0000000713600000, 0x0000000713600000|100%| S|CS|TAMS 0x0000000713400000, 0x0000000713400000| Complete -| 155|0x0000000713600000, 0x0000000713800000, 0x0000000713800000|100%| S|CS|TAMS 0x0000000713600000, 0x0000000713600000| Complete -| 156|0x0000000713800000, 0x0000000713a00000, 0x0000000713a00000|100%| S|CS|TAMS 0x0000000713800000, 0x0000000713800000| Complete -| 157|0x0000000713a00000, 0x0000000713b14ee8, 0x0000000713c00000| 54%| E| |TAMS 0x0000000713a00000, 0x0000000713a00000| Complete -| 158|0x0000000713c00000, 0x0000000713e00000, 0x0000000713e00000|100%| E|CS|TAMS 0x0000000713c00000, 0x0000000713c00000| Complete -| 159|0x0000000713e00000, 0x0000000714000000, 0x0000000714000000|100%| E|CS|TAMS 0x0000000713e00000, 0x0000000713e00000| Complete -| 160|0x0000000714000000, 0x0000000714200000, 0x0000000714200000|100%| E|CS|TAMS 0x0000000714000000, 0x0000000714000000| Complete -| 161|0x0000000714200000, 0x0000000714400000, 0x0000000714400000|100%| E|CS|TAMS 0x0000000714200000, 0x0000000714200000| Complete -| 334|0x0000000729c00000, 0x0000000729e00000, 0x0000000729e00000|100%| E|CS|TAMS 0x0000000729c00000, 0x0000000729c00000| Complete -| 335|0x0000000729e00000, 0x000000072a000000, 0x000000072a000000|100%| E|CS|TAMS 0x0000000729e00000, 0x0000000729e00000| Complete -| 440|0x0000000737000000, 0x0000000737200000, 0x0000000737200000|100%| E|CS|TAMS 0x0000000737000000, 0x0000000737000000| Complete -| 441|0x0000000737200000, 0x0000000737400000, 0x0000000737400000|100%| E|CS|TAMS 0x0000000737200000, 0x0000000737200000| Complete -| 442|0x0000000737400000, 0x0000000737600000, 0x0000000737600000|100%| E|CS|TAMS 0x0000000737400000, 0x0000000737400000| Complete -| 767|0x000000075fe00000, 0x0000000760000000, 0x0000000760000000|100%| E|CS|TAMS 0x000000075fe00000, 0x000000075fe00000| Complete -|2046|0x00000007ffc00000, 0x00000007ffd74000, 0x00000007ffe00000| 72%|OA| |TAMS 0x00000007ffd74000, 0x00000007ffd74000| Untracked -|2047|0x00000007ffe00000, 0x00000007ffe74000, 0x0000000800000000| 22%|CA| |TAMS 0x00000007ffe74000, 0x00000007ffe74000| Untracked - -Card table byte_map: [0x0000000122604000,0x0000000122e04000] _byte_map_base: 0x000000011ee04000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x0000000104f136f0, (CMBitMap*) 0x0000000104f13730 - Prev Bits: [0x0000000123604000, 0x0000000127604000) - Next Bits: [0x0000000127604000, 0x000000012b604000) - -Polling page: 0x0000000104824000 - -Metaspace: - -Usage: - Non-class: 130.16 MB used. - Class: 17.80 MB used. - Both: 147.95 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 130.81 MB ( 68%) committed, 3 nodes. - Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 149.19 MB ( 12%) committed. - -Chunk freelists: - Non-Class: 12.97 MB - Class: 13.67 MB - Both: 26.64 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 164.31 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 14. -num_arena_births: 2088. -num_arena_deaths: 0. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2384. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 14. -num_chunks_taken_from_freelist: 7782. -num_chunk_merges: 11. -num_chunk_splits: 5822. -num_chunks_enlarged: 4401. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=8986Kb max_used=8986Kb free=110182Kb - bounds [0x000000011a9e8000, 0x000000011b2c8000, 0x0000000121e48000] -CodeHeap 'profiled nmethods': size=119152Kb used=19697Kb max_used=20086Kb free=99454Kb - bounds [0x0000000112e48000, 0x00000001141e8000, 0x000000011a2a4000] -CodeHeap 'non-nmethods': size=7440Kb used=3323Kb max_used=3467Kb free=4116Kb - bounds [0x000000011a2a4000, 0x000000011a624000, 0x000000011a9e8000] - total_blobs=11917 nmethods=10855 adapters=973 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 20.973 Thread 0x0000000b668e6400 14759 3 org.apache.comet.serde.Config$ConfigMap$Builder::putEntries (87 bytes) -Event: 20.973 Thread 0x0000000b66930000 nmethod 14754 0x00000001138b2b10 code [0x00000001138b2cc0, 0x00000001138b2eb8] -Event: 20.973 Thread 0x0000000b66930000 14763 3 com.google.protobuf.MapEntry::writeTo (26 bytes) -Event: 20.973 Thread 0x0000000b66930000 nmethod 14763 0x00000001138b2610 code [0x00000001138b27c0, 0x00000001138b29b8] -Event: 20.973 Thread 0x0000000b66930000 14764 3 com.google.protobuf.MapEntryLite::writeTo (38 bytes) -Event: 20.973 Thread 0x0000000b668e6400 nmethod 14759 0x00000001138b1b90 code [0x00000001138b1dc0, 0x00000001138b2348] -Event: 20.973 Thread 0x0000000b668e6400 14766 3 org.apache.comet.ConfigEntry::key (13 bytes) -Event: 20.973 Thread 0x0000000b668e6400 nmethod 14766 0x00000001138b1710 code [0x00000001138b18c0, 0x00000001138b1a48] -Event: 20.973 Thread 0x0000000b66930000 nmethod 14764 0x00000001138b1110 code [0x00000001138b1300, 0x00000001138b1568] -Event: 20.973 Thread 0x0000000b66930000 14762 3 org.apache.logging.log4j.message.AbstractMessageFactory::newMessage (17 bytes) -Event: 20.973 Thread 0x0000000b668e6400 14765 3 org.apache.logging.log4j.message.SimpleMessage:: (23 bytes) -Event: 20.973 Thread 0x0000000b668e6400 nmethod 14765 0x00000001138b0b90 code [0x00000001138b0d40, 0x00000001138b0fc8] -Event: 20.973 Thread 0x0000000b668e7600 nmethod 14758 0x00000001138afc10 code [0x00000001138afec0, 0x00000001138b0778] -Event: 20.973 Thread 0x0000000b66930000 nmethod 14762 0x00000001138af690 code [0x00000001138af840, 0x00000001138afa78] -Event: 20.973 Thread 0x0000000b65684000 nmethod 14756 0x0000000113dd5f10 code [0x0000000113dd65c0, 0x0000000113dd99e8] -Event: 20.973 Thread 0x0000000b668e7600 14767 3 scala.collection.immutable.Map$EmptyMap$::updated (21 bytes) -Event: 20.973 Thread 0x0000000b668e7600 nmethod 14767 0x00000001138af110 code [0x00000001138af2c0, 0x00000001138af4f8] -Event: 20.974 Thread 0x0000000b65684000 14768 3 java.util.stream.ReferencePipeline$2$1::begin (13 bytes) -Event: 20.974 Thread 0x0000000b668e6400 14769 3 java.util.Optional::isPresent (13 bytes) -Event: 20.974 Thread 0x0000000b65684000 nmethod 14768 0x00000001138aec90 code [0x00000001138aee40, 0x00000001138af048] - -GC Heap History (20 events): -Event: 7.381 GC heap before -{Heap before GC invocations=27 (full 0): - garbage-first heap total 348160K, used 283997K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 66 young (135168K), 2 survivors (4096K) - Metaspace used 140795K, committed 141760K, reserved 1179648K - class space used 16877K, committed 17344K, reserved 1048576K -} -Event: 7.382 GC heap after -{Heap after GC invocations=28 (full 0): - garbage-first heap total 348160K, used 152280K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 140795K, committed 141760K, reserved 1179648K - class space used 16877K, committed 17344K, reserved 1048576K -} -Event: 7.558 GC heap before -{Heap before GC invocations=28 (full 0): - garbage-first heap total 348160K, used 289496K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 68 young (139264K), 3 survivors (6144K) - Metaspace used 147451K, committed 148544K, reserved 1179648K - class space used 17566K, committed 18048K, reserved 1048576K -} -Event: 7.559 GC heap after -{Heap after GC invocations=29 (full 0): - garbage-first heap total 348160K, used 154116K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 147451K, committed 148544K, reserved 1179648K - class space used 17566K, committed 18048K, reserved 1048576K -} -Event: 7.889 GC heap before -{Heap before GC invocations=29 (full 0): - garbage-first heap total 348160K, used 289284K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 70 young (143360K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 7.891 GC heap after -{Heap after GC invocations=30 (full 0): - garbage-first heap total 348160K, used 154509K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.181 GC heap before -{Heap before GC invocations=30 (full 0): - garbage-first heap total 348160K, used 291725K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 71 young (145408K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.183 GC heap after -{Heap after GC invocations=31 (full 0): - garbage-first heap total 348160K, used 154662K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.466 GC heap before -{Heap before GC invocations=31 (full 0): - garbage-first heap total 348160K, used 295974K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 73 young (149504K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.467 GC heap after -{Heap after GC invocations=32 (full 0): - garbage-first heap total 348160K, used 154558K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.753 GC heap before -{Heap before GC invocations=32 (full 0): - garbage-first heap total 348160K, used 297918K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 74 young (151552K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 8.754 GC heap after -{Heap after GC invocations=33 (full 0): - garbage-first heap total 348160K, used 154400K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 9.049 GC heap before -{Heap before GC invocations=33 (full 0): - garbage-first heap total 348160K, used 299808K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 75 young (153600K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 9.050 GC heap after -{Heap after GC invocations=34 (full 0): - garbage-first heap total 348160K, used 154646K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148885K, committed 150016K, reserved 1245184K - class space used 17782K, committed 18304K, reserved 1048576K -} -Event: 20.862 GC heap before -{Heap before GC invocations=34 (full 0): - garbage-first heap total 348160K, used 302102K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 75 young (153600K), 4 survivors (8192K) - Metaspace used 149316K, committed 150400K, reserved 1245184K - class space used 17855K, committed 18368K, reserved 1048576K -} -Event: 20.864 GC heap after -{Heap after GC invocations=35 (full 0): - garbage-first heap total 348160K, used 154622K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 149316K, committed 150400K, reserved 1245184K - class space used 17855K, committed 18368K, reserved 1048576K -} -Event: 20.887 GC heap before -{Heap before GC invocations=35 (full 0): - garbage-first heap total 348160K, used 160766K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 6 young (12288K), 4 survivors (8192K) - Metaspace used 149347K, committed 150464K, reserved 1245184K - class space used 17855K, committed 18368K, reserved 1048576K -} -Event: 20.889 GC heap after -{Heap after GC invocations=36 (full 0): - garbage-first heap total 348160K, used 154628K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 149347K, committed 150464K, reserved 1245184K - class space used 17855K, committed 18368K, reserved 1048576K -} -Event: 20.938 GC heap before -{Heap before GC invocations=36 (full 0): - garbage-first heap total 348160K, used 168964K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 11 young (22528K), 4 survivors (8192K) - Metaspace used 150217K, committed 151360K, reserved 1245184K - class space used 17979K, committed 18496K, reserved 1048576K -} -Event: 20.940 GC heap after -{Heap after GC invocations=37 (full 0): - garbage-first heap total 348160K, used 155038K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 150217K, committed 151360K, reserved 1245184K - class space used 17979K, committed 18496K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.443 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li -Event: 3.444 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.444 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.423 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib -Event: 5.936 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 5.936 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 5.936 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 5.936 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 5.936 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 5.936 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 5.936 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.936 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 5.936 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 5.936 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.936 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 5.936 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 5.937 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 5.937 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 6.131 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-11281734455283120293.dylib -Event: 6.834 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-2574e9d4-69d6-4911-a16f-7c6239f501cb-libsnappyjava.dylib - -Deoptimization events (20 events): -Event: 20.964 Thread 0x0000000b661e1800 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b21a33c relative=0x000000000000013c -Event: 20.964 Thread 0x0000000b661e2400 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 20.964 Thread 0x0000000b661e1e00 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017cdc8ef0 mode 2 -Event: 20.964 Thread 0x0000000b661e1800 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 20.964 Thread 0x0000000b661e2400 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017d1d4ff0 -Event: 20.964 Thread 0x0000000b661e1800 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017c9bcff0 -Event: 20.964 Thread 0x0000000b661e2a00 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011b21a33c relative=0x000000000000013c -Event: 20.964 Thread 0x0000000b661e2a00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011b21a33c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 20.964 Thread 0x0000000b661e2400 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017d1d4ef0 mode 2 -Event: 20.964 Thread 0x0000000b661e2a00 DEOPT PACKING pc=0x000000011b21a33c sp=0x000000017d5e0ff0 -Event: 20.964 Thread 0x0000000b661e1800 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c9bcef0 mode 2 -Event: 20.964 Thread 0x0000000b661e2a00 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017d5e0ef0 mode 2 -Event: 20.964 Thread 0x0000000b661e1200 Uncommon trap: trap_request=0xffffffde fr.pc=0x000000011af9baa8 relative=0x00000000000000e8 -Event: 20.964 Thread 0x0000000b661e1200 Uncommon trap: reason=class_check action=maybe_recompile pc=0x000000011af9baa8 method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 20.964 Thread 0x0000000b661e1200 DEOPT PACKING pc=0x000000011af9baa8 sp=0x000000017c5b0f70 -Event: 20.964 Thread 0x0000000b661e1200 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c5b0ef0 mode 2 -Event: 20.968 Thread 0x0000000b661e1800 Uncommon trap: trap_request=0xffffffc6 fr.pc=0x000000011acf1e10 relative=0x0000000000000810 -Event: 20.968 Thread 0x0000000b661e1800 Uncommon trap: reason=bimorphic_or_optimized_type_check action=maybe_recompile pc=0x000000011acf1e10 method=scala.collection.mutable.HashTable.findEntry0(Ljava/lang/Object;I)Lscala/collection/mutable/HashEntry; @ 63 c2 -Event: 20.968 Thread 0x0000000b661e1800 DEOPT PACKING pc=0x000000011acf1e10 sp=0x000000017c9bc850 -Event: 20.968 Thread 0x0000000b661e1800 DEOPT UNPACKING pc=0x000000011a2eb91c sp=0x000000017c9bc6a0 mode 2 - -Classes loaded (20 events): -Event: 7.527 Loading class java/lang/Class$EnclosingMethodInfo -Event: 7.527 Loading class java/lang/Class$EnclosingMethodInfo done -Event: 18.268 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper -Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$Unmapper -Event: 18.269 Loading class jdk/internal/access/foreign/UnmapperProxy -Event: 18.269 Loading class jdk/internal/access/foreign/UnmapperProxy done -Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$Unmapper done -Event: 18.269 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done -Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask -Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask -Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done -Event: 18.271 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done -Event: 18.271 Loading class java/util/function/LongBinaryOperator -Event: 18.271 Loading class java/util/function/LongBinaryOperator done -Event: 20.844 Loading class sun/nio/ch/ChannelInputStream -Event: 20.844 Loading class sun/nio/ch/ChannelInputStream done -Event: 20.903 Loading class java/math/BigDecimal$StringBuilderHelper -Event: 20.903 Loading class java/math/BigDecimal$StringBuilderHelper done -Event: 20.969 Loading class java/nio/channels/Channels$ReadableByteChannelImpl -Event: 20.969 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 20.964 Thread 0x0000000b661e1800 Exception (0x0000000714144c00) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.965 Thread 0x0000000b661e2a00 Exception (0x0000000714389228) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.965 Thread 0x0000000b661e2400 Exception (0x0000000729e6e3d8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.965 Thread 0x0000000b661e1200 Exception (0x00000007140f2820) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.965 Thread 0x0000000b661e1800 Exception (0x0000000714148218) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ebdf18) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000714106a28) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000714195898) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x000000071439d3f8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e82960) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000713f0c988) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ec1ef0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000714199670) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x00000007143a11d0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e86738) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1e00 Exception (0x0000000713ec57d0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2400 Exception (0x0000000729e89fc8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1200 Exception (0x0000000713f10218) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e1800 Exception (0x0000000713f93530) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 20.967 Thread 0x0000000b661e2a00 Exception (0x0000000713c00830) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] - -VM Operations (20 events): -Event: 8.466 Executing VM operation: G1CollectForAllocation -Event: 8.467 Executing VM operation: G1CollectForAllocation done -Event: 8.753 Executing VM operation: G1CollectForAllocation -Event: 8.754 Executing VM operation: G1CollectForAllocation done -Event: 9.049 Executing VM operation: G1CollectForAllocation -Event: 9.050 Executing VM operation: G1CollectForAllocation done -Event: 12.065 Executing VM operation: Cleanup -Event: 12.065 Executing VM operation: Cleanup done -Event: 19.099 Executing VM operation: Cleanup -Event: 19.099 Executing VM operation: Cleanup done -Event: 20.104 Executing VM operation: Cleanup -Event: 20.104 Executing VM operation: Cleanup done -Event: 20.862 Executing VM operation: G1TryInitiateConcMark -Event: 20.864 Executing VM operation: G1TryInitiateConcMark done -Event: 20.887 Executing VM operation: G1TryInitiateConcMark -Event: 20.889 Executing VM operation: G1TryInitiateConcMark done -Event: 20.929 Executing VM operation: ICBufferFull -Event: 20.929 Executing VM operation: ICBufferFull done -Event: 20.938 Executing VM operation: G1TryInitiateConcMark -Event: 20.940 Executing VM operation: G1TryInitiateConcMark done - -Memory protections (20 events): -Event: 7.391 Protecting memory [0x000000017f240000,0x000000017f24c000] with protection modes 0 -Event: 7.391 Protecting memory [0x000000017f64c000,0x000000017f658000] with protection modes 0 -Event: 9.328 Protecting memory [0x000000016fac8000,0x000000016fad4000] with protection modes 3 -Event: 9.328 Protecting memory [0x000000016f8bc000,0x000000016f8c8000] with protection modes 3 -Event: 9.328 Protecting memory [0x000000016ea74000,0x000000016ea80000] with protection modes 3 -Event: 9.331 Protecting memory [0x000000016f098000,0x000000016f0a4000] with protection modes 3 -Event: 14.336 Protecting memory [0x000000016ee8c000,0x000000016ee98000] with protection modes 3 -Event: 14.336 Protecting memory [0x000000016e868000,0x000000016e874000] with protection modes 3 -Event: 14.336 Protecting memory [0x000000016ec80000,0x000000016ec8c000] with protection modes 3 -Event: 20.846 Protecting memory [0x000000016e868000,0x000000016e874000] with protection modes 0 -Event: 20.856 Protecting memory [0x000000016ea74000,0x000000016ea80000] with protection modes 0 -Event: 20.856 Protecting memory [0x000000016ec80000,0x000000016ec8c000] with protection modes 0 -Event: 20.856 Protecting memory [0x000000016ee8c000,0x000000016ee98000] with protection modes 0 -Event: 20.874 Protecting memory [0x000000016f098000,0x000000016f0a4000] with protection modes 0 -Event: 20.893 Protecting memory [0x000000016f8bc000,0x000000016f8c8000] with protection modes 0 -Event: 20.941 Protecting memory [0x000000030041c000,0x0000000300428000] with protection modes 0 -Event: 20.941 Protecting memory [0x0000000300828000,0x0000000300834000] with protection modes 0 -Event: 20.941 Protecting memory [0x0000000300e58000,0x0000000300e64000] with protection modes 0 -Event: 20.942 Protecting memory [0x0000000301264000,0x0000000301270000] with protection modes 0 -Event: 20.942 Protecting memory [0x0000000301670000,0x000000030167c000] with protection modes 0 - -Nmethod flushes (20 events): -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416de90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416ec10 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416f310 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011416fb90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114170d90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114171c10 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114172d90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114173890 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114175990 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114176d10 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114177f90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114178410 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114178d90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114179810 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114179c90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011417af90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x000000011417f610 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114180a90 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114181590 -Event: 7.676 Thread 0x0000000b668e6a00 flushing nmethod 0x0000000114182710 - -Events (20 events): -Event: 7.391 Thread 0x0000000b64460c00 Thread added: 0x0000000b64461200 -Event: 7.391 Thread 0x0000000b64461200 Thread added: 0x0000000b64461800 -Event: 9.328 Thread 0x0000000b661e3600 Thread exited: 0x0000000b661e3600 -Event: 9.328 Thread 0x0000000b66930000 Thread exited: 0x0000000b66930000 -Event: 9.328 Thread 0x0000000b65684000 Thread exited: 0x0000000b65684000 -Event: 9.331 Thread 0x0000000b6619b000 Thread exited: 0x0000000b6619b000 -Event: 14.336 Thread 0x0000000b65ca3600 Thread exited: 0x0000000b65ca3600 -Event: 14.336 Thread 0x0000000b65ca0600 Thread exited: 0x0000000b65ca0600 -Event: 14.336 Thread 0x0000000b668e7600 Thread exited: 0x0000000b668e7600 -Event: 20.846 Thread 0x0000000b668e6400 Thread added: 0x0000000b6451f600 -Event: 20.856 Thread 0x0000000b668e6400 Thread added: 0x0000000b668e7600 -Event: 20.856 Thread 0x0000000b668e6400 Thread added: 0x0000000b65684000 -Event: 20.856 Thread 0x0000000b668e7600 Thread added: 0x0000000b66930000 -Event: 20.874 Thread 0x0000000b6451f600 Thread added: 0x0000000b66b0e400 -Event: 20.893 Thread 0x0000000b65095800 Thread added: 0x0000000b66198000 -Event: 20.941 Thread 0x0000000b65529200 Thread added: 0x0000000b661e3600 -Event: 20.941 Thread 0x0000000b661e3600 Thread added: 0x0000000b6451c600 -Event: 20.941 Thread 0x0000000b661e3600 Thread added: 0x0000000b64463600 -Event: 20.941 Thread 0x0000000b64463600 Thread added: 0x0000000b63d64000 -Event: 20.942 Thread 0x0000000b63d64000 Thread added: 0x0000000b63d64600 - - -Dynamic libraries: -0x000000010483c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x000000010a304000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x0000000104864000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x0000000104894000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x00000001048e4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x0000000104980000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x00000001049a8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x00000001049c8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x00000001049ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x000000010496c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x0000000104a08000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x0000000104a1c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x0000000109e7c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000149dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-14395648754114117926.dylib -0x0000000109f4c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-11281734455283120293.dylib -0x0000000109f78000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-2574e9d4-69d6-4911-a16f-7c6239f501cb-libsnappyjava.dylib - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:39 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 8.72 7.58 5.42 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(863632k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. diff --git a/spark/hs_err_pid59157.log b/spark/hs_err_pid59157.log deleted file mode 100644 index 8660ac9ed5..0000000000 --- a/spark/hs_err_pid59157.log +++ /dev/null @@ -1,3785 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x0000000108ca9530, pid=59157, tid=29447 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 08:59:18 2026 MDT elapsed time: 44.767401 seconds (0d 0h 0m 44s) - ---------------- T H R E A D --------------- - -Current thread (0x0000000c67a9a400): JavaThread "Executor task launch worker for task 0.0 in stage 268.0 (TID 1092)" daemon [_thread_in_vm, id=29447, stack(0x000000017de60000,0x000000017e263000)] - -Stack: [0x000000017de60000,0x000000017e263000], sp=0x000000017e25a520, free space=4073k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 -C [libcomet-1214774261196833962.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 -C [libcomet-1214774261196833962.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc -C [libcomet-1214774261196833962.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 -C [libcomet-1214774261196833962.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 -C [libcomet-1214774261196833962.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c -C [libcomet-1214774261196833962.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 -C [libcomet-1214774261196833962.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 -C [libcomet-1214774261196833962.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 -C [libcomet-1214774261196833962.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 -C [libcomet-1214774261196833962.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 -C [libcomet-1214774261196833962.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 -C [libcomet-1214774261196833962.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 -C [libcomet-1214774261196833962.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c -C [libcomet-1214774261196833962.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 -C [libcomet-1214774261196833962.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 -C [libcomet-1214774261196833962.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c -C [libcomet-1214774261196833962.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 -C [libcomet-1214774261196833962.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 -C [libcomet-1214774261196833962.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 -C [libcomet-1214774261196833962.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c -C [libcomet-1214774261196833962.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 -C [libcomet-1214774261196833962.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 -C [libcomet-1214774261196833962.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 -C [libcomet-1214774261196833962.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c -C [libcomet-1214774261196833962.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c -C [libcomet-1214774261196833962.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-1214774261196833962.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-1214774261196833962.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-1214774261196833962.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-1214774261196833962.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-1214774261196833962.dylib+0x106628] __rust_try+0x20 -C [libcomet-1214774261196833962.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-1214774261196833962.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-1214774261196833962.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-1214774261196833962.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -J 17175 org.apache.comet.Native.executePlan(IIJ[J[J)J (0 bytes) @ 0x000000011984a554 [0x000000011984a4c0+0x0000000000000094] -J 18191 c1 org.apache.comet.vector.NativeUtil.allocateArrowStructs(I)Lscala/Tuple2; (69 bytes) @ 0x0000000112c2d368 [0x0000000112c2cb80+0x00000000000007e8] -J 18196 c1 org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option; (296 bytes) @ 0x0000000112c30e90 [0x0000000112c2fdc0+0x00000000000010d0] -J 24291 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option; (35 bytes) @ 0x00000001135ce88c [0x00000001135ce380+0x000000000000050c] -J 24290 c1 org.apache.comet.CometExecIterator$$Lambda$3979+0x000000c8021a53e8.apply()Ljava/lang/Object; (12 bytes) @ 0x00000001135cdf6c [0x00000001135cdec0+0x00000000000000ac] -J 24159 c1 org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object; (136 bytes) @ 0x0000000113589430 [0x00000001135891c0+0x0000000000000270] -J 24158 c1 org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option; (494 bytes) @ 0x000000011358b4f0 [0x000000011358aa80+0x0000000000000a70] -J 24297 c1 org.apache.comet.CometExecIterator.hasNext()Z (261 bytes) @ 0x00000001135d2384 [0x00000001135d1d80+0x0000000000000604] -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -J 33848 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus; (309 bytes) @ 0x0000000112892fa4 [0x0000000112890ac0+0x00000000000024e4] -J 33847 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object; (15 bytes) @ 0x0000000112fbca94 [0x0000000112fbc9c0+0x00000000000000d4] -J 30108 c1 org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object; (139 bytes) @ 0x0000000113ecd11c [0x0000000113eccfc0+0x000000000000015c] -J 26867 c1 org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object; (628 bytes) @ 0x0000000113819eb0 [0x00000001138183c0+0x0000000000001af0] -J 29617 c1 org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object; (102 bytes) @ 0x0000000113decc34 [0x0000000113dec940+0x00000000000002f4] -J 26396 c1 org.apache.spark.executor.Executor$TaskRunner$$Lambda$2518+0x000000c801df1868.apply()Ljava/lang/Object; (12 bytes) @ 0x000000011374af6c [0x000000011374aec0+0x00000000000000ac] -J 17578 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (402 bytes) @ 0x0000000112b3ec74 [0x0000000112b3eb00+0x0000000000000174] -J 21288 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000113036c14 [0x0000000113036b40+0x00000000000000d4] -J 17575 c1 org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000112b36b84 [0x0000000112b36ac0+0x00000000000000c4] -J 29509 c1 org.apache.spark.executor.Executor$TaskRunner.run()V (3920 bytes) @ 0x0000000113da7838 [0x0000000113da2e00+0x0000000000004a38] -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -J 17175 org.apache.comet.Native.executePlan(IIJ[J[J)J (0 bytes) @ 0x000000011984a554 [0x000000011984a4c0+0x0000000000000094] -J 24154 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J (34 bytes) @ 0x000000011358720c [0x00000001135870c0+0x000000000000014c] -J 24294 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object; (23 bytes) @ 0x00000001135cf684 [0x00000001135cf600+0x0000000000000084] -J 24296 c1 org.apache.comet.CometExecIterator$$Lambda$3982+0x000000c8021a67b8.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object; (20 bytes) @ 0x00000001135cfdec [0x00000001135cfb40+0x00000000000002ac] -J 18196 c1 org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option; (296 bytes) @ 0x0000000112c30e90 [0x0000000112c2fdc0+0x00000000000010d0] -J 24291 c1 org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option; (35 bytes) @ 0x00000001135ce88c [0x00000001135ce380+0x000000000000050c] -J 24290 c1 org.apache.comet.CometExecIterator$$Lambda$3979+0x000000c8021a53e8.apply()Ljava/lang/Object; (12 bytes) @ 0x00000001135cdf6c [0x00000001135cdec0+0x00000000000000ac] -J 24159 c1 org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object; (136 bytes) @ 0x0000000113589430 [0x00000001135891c0+0x0000000000000270] -J 24158 c1 org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option; (494 bytes) @ 0x000000011358b4f0 [0x000000011358aa80+0x0000000000000a70] -J 24297 c1 org.apache.comet.CometExecIterator.hasNext()Z (261 bytes) @ 0x00000001135d2384 [0x00000001135d1d80+0x0000000000000604] -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -J 33848 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus; (309 bytes) @ 0x0000000112892fa4 [0x0000000112890ac0+0x00000000000024e4] -J 33847 c1 org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object; (15 bytes) @ 0x0000000112fbca94 [0x0000000112fbc9c0+0x00000000000000d4] -J 30108 c1 org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object; (139 bytes) @ 0x0000000113ecd11c [0x0000000113eccfc0+0x000000000000015c] -J 26867 c1 org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object; (628 bytes) @ 0x0000000113819eb0 [0x00000001138183c0+0x0000000000001af0] -J 29617 c1 org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object; (102 bytes) @ 0x0000000113decc34 [0x0000000113dec940+0x00000000000002f4] -J 26396 c1 org.apache.spark.executor.Executor$TaskRunner$$Lambda$2518+0x000000c801df1868.apply()Ljava/lang/Object; (12 bytes) @ 0x000000011374af6c [0x000000011374aec0+0x00000000000000ac] -J 17578 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (402 bytes) @ 0x0000000112b3ec74 [0x0000000112b3eb00+0x0000000000000174] -J 21288 c1 org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000113036c14 [0x0000000113036b40+0x00000000000000d4] -J 17575 c1 org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object; (18 bytes) @ 0x0000000112b36b84 [0x0000000112b36ac0+0x00000000000000c4] -J 29509 c1 org.apache.spark.executor.Executor$TaskRunner.run()V (3920 bytes) @ 0x0000000113da7838 [0x0000000113da2e00+0x0000000000004a38] -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 - -Registers: - x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000c67b00930 - x4=0x000000017e25a640 x5=0x0000000c67a9a400 x6=0x0000000c74e14300 x7=0xfffff0003ffff800 - x8=0x000000c801e60a98 x9=0x0000000000000000 x10=0x000000c800000000 x11=0x0000000000000004 -x12=0x000000013471cde0 x13=0x0000000000000000 x14=0xfffffffffffdb160 x15=0x0000000c69174000 -x16=0x0000000000000040 x17=0x0000000c690d2d10 x18=0x0000000000000000 x19=0x0000000c67a9a400 -x20=0x000000017e25a670 x21=0x000000017e25a640 x22=0x0000000c67054918 x23=0x0000000733506718 -x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000c67a9dac0 x27=0x0000000702bbdd88 -x28=0x0000000000000001 fp=0x000000017e25a630 lr=0x0000000108ca952c sp=0x000000017e25a520 -pc=0x0000000108ca9530 cpsr=0x0000000000001000 - -Register to memory mapping: - - x0=0x0 is NULL - x1=0x0000000000000007 is an unknown value - x2=0x0000000000000001 is an unknown value - x3=0x0000000c67b00930 points into unknown readable memory: 0x000000013471cde0 | e0 cd 71 34 01 00 00 00 - x4=0x000000017e25a640 is pointing into the stack for thread: 0x0000000c67a9a400 - x5=0x0000000c67a9a400 is a thread - x6=0x0000000c74e14300 points into unknown readable memory: 0x0000000c66005200 | 00 52 00 66 0c 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=0x000000c801e60a98 is pointing into metadata - x9=0x0 is NULL -x10=0x000000c800000000 is pointing into metadata -x11=0x0000000000000004 is an unknown value -x12={method} {0x000000013471cde0} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' -x13=0x0 is NULL -x14=0xfffffffffffdb160 is an unknown value -x15=0x0000000c69174000 points into unknown readable memory: 0xffffffffffffffff | ff ff ff ff ff ff ff ff -x16=0x0000000000000040 is an unknown value -x17=0x0000000c690d2d10 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff -x18=0x0 is NULL -x19=0x0000000c67a9a400 is a thread -x20=0x000000017e25a670 is pointing into the stack for thread: 0x0000000c67a9a400 -x21=0x000000017e25a640 is pointing into the stack for thread: 0x0000000c67a9a400 -x22=0x0000000c67054918 points into unknown readable memory: 0x0000000733506718 | 18 67 50 33 07 00 00 00 -x23=0x0000000733506718 is an oop: org.apache.comet.CometHandleBatchIterator -{0x0000000733506718} - klass: 'org/apache/comet/CometHandleBatchIterator' - - ---- fields (total size 2 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x00000007334fb620} (e669f6c4) -x24=0x0000000000000001 is an unknown value -x25=0x0 is NULL -x26=0x0000000c67a9dac0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x27=0x0000000702bbdd88 is an oop: [Z -{0x0000000702bbdd88} - klass: {type array bool} - - length: 9 -x28=0x0000000000000001 is an unknown value - - -Top of Stack: (sp=0x000000017e25a520) -0x000000017e25a520: 0000000c736e4000 0000000000000014 -0x000000017e25a530: 0000000c70134200 0000000000000014 -0x000000017e25a540: 0000000000000001 0000000000000000 -0x000000017e25a550: 000000017e25a587 0000000000000000 -0x000000017e25a560: 000000017e25a5a0 00000001587b40d0 -0x000000017e25a570: 0000000000000014 0000000c65f094d8 -0x000000017e25a580: 0000000000000014 0000000c65f094d8 -0x000000017e25a590: 0000000c65f094d8 0000000000000014 -0x000000017e25a5a0: 000000017e25a630 00000001587e3f58 -0x000000017e25a5b0: 0000000000000063 000000017e25a640 -0x000000017e25a5c0: 000000017e25a630 0000000108cc5f24 -0x000000017e25a5d0: 95e350fae7f500a8 0000000c67a9a400 -0x000000017e25a5e0: 0000000c67a9a400 0000000702bbdd88 -0x000000017e25a5f0: 0000000702b18548 0000000000000000 -0x000000017e25a600: 0000000c67a9a6b0 0000000000000008 -0x000000017e25a610: 0000000c6600a0a8 0000000c67b00930 -0x000000017e25a620: 0000000000000001 0000000c67a9a400 -0x000000017e25a630: 000000017e25a6e0 0000000108cad60c -0x000000017e25a640: 00000001093ee6b8 000000c80045a1d0 -0x000000017e25a650: 000000010000000a 0000000000000014 -0x000000017e25a660: 0000000000000000 0000000000000008 -0x000000017e25a670: 000000070000000a 000000073350ef48 -0x000000017e25a680: 0000000c67a9a400 0000000000000000 -0x000000017e25a690: 000000017e25a6e0 0000000151eb8c58 -0x000000017e25a6a0: 0000000702b18548 000000000000002f -0x000000017e25a6b0: 000000070483d960 0000000c638ed780 -0x000000017e25a6c0: 000000073350ef48 000000073350ef48 -0x000000017e25a6d0: 000000070483d960 000000000000002f -0x000000017e25a6e0: 000000017e25c2a0 0000000151e7d560 -0x000000017e25a6f0: 0000000000000000 000000017e25adc0 -0x000000017e25a700: 0000000c74935338 0000000000000038 -0x000000017e25a710: 0000000000000008 0000000159025fc8 - -Instructions: (pc=0x0000000108ca9530) -0x0000000108ca9430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 -0x0000000108ca9440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 -0x0000000108ca9450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 -0x0000000108ca9460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 -0x0000000108ca9470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 -0x0000000108ca9480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 -0x0000000108ca9490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 -0x0000000108ca94a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 -0x0000000108ca94b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa -0x0000000108ca94c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 -0x0000000108ca94d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 -0x0000000108ca94e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 -0x0000000108ca94f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a -0x0000000108ca9500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa -0x0000000108ca9510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 -0x0000000108ca9520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa -0x0000000108ca9530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 -0x0000000108ca9540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 -0x0000000108ca9550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 -0x0000000108ca9560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 -0x0000000108ca9570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 -0x0000000108ca9580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 -0x0000000108ca9590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa -0x0000000108ca95a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 -0x0000000108ca95b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 -0x0000000108ca95c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 -0x0000000108ca95d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 -0x0000000108ca95e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 -0x0000000108ca95f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa -0x0000000108ca9600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 -0x0000000108ca9610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 -0x0000000108ca9620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x0000000c736e4000 points into unknown readable memory: 0x0000000000000014 | 14 00 00 00 00 00 00 00 -stack at sp + 1 slots: 0x0000000000000014 is an unknown value -stack at sp + 2 slots: 0x0000000c70134200 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -stack at sp + 3 slots: 0x0000000000000014 is an unknown value -stack at sp + 4 slots: 0x0000000000000001 is an unknown value -stack at sp + 5 slots: 0x0 is NULL -stack at sp + 6 slots: 0x000000017e25a587 is pointing into the stack for thread: 0x0000000c67a9a400 -stack at sp + 7 slots: 0x0 is NULL - - -Compiled method (n/a) 44941 17175 n 0 org.apache.comet.Native::executePlan (native) - total in heap [0x000000011984a310,0x000000011984a688] = 888 - relocation [0x000000011984a468,0x000000011984a490] = 40 - main code [0x000000011984a4c0,0x000000011984a678] = 440 - oops [0x000000011984a678,0x000000011984a680] = 8 - metadata [0x000000011984a680,0x000000011984a688] = 8 - -[Constant Pool (empty)] - -[MachCode] -[Entry Point] - # {method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native' - # this: c_rarg1:c_rarg1 - = 'org/apache/comet/Native' - # parm0: c_rarg2 = int - # parm1: c_rarg3 = int - # parm2: c_rarg4:c_rarg4 - = long - # parm3: c_rarg5:c_rarg5 - = '[J' - # parm4: c_rarg6:c_rarg6 - = '[J' - # [sp+0x60] (sp of caller) - 0x000000011984a4c0: 2808 40b9 | 3f01 086b | 4000 0054 - - 0x000000011984a4cc: ; {runtime_call ic_miss_stub} - 0x000000011984a4cc: cd09 be17 -[Verified Entry Point] - 0x000000011984a4d0: 1f20 03d5 | e953 40d1 | 3f01 00f9 | fd7b bfa9 | fd03 0091 | ff43 01d1 | e617 00f9 | df00 1feb - 0x000000011984a4f0: e6a3 0091 | e603 869a | e513 00f9 | bf00 1feb | e583 0091 | e503 859a | e103 00f9 | 3f00 1feb - 0x000000011984a510: e103 00d1 | e103 819a | e801 0010 | 884f 01f9 | e803 0091 | 884b 01f9 - - 0x000000011984a528: ; {external_word} - 0x000000011984a528: c8df f7b0 | 08a9 5339 | 8806 0035 | 80c3 0a91 | e803 7eb2 | 89e3 0c91 | 28fd 9f88 - - 0x000000011984a544: ; {runtime_call Java_org_apache_comet_Native_executePlan} - 0x000000011984a544: 88c1 93d2 | a83f aaf2 | 2800 c0f2 | 0001 3fd6 | a800 80d2 | 883b 03b9 | bf3b 03d5 | 8803 0d91 - 0x000000011984a564: 08fd dfc8 | bf03 08eb | a803 0054 | 882b 43b9 | 6803 0035 | e803 7db2 | 89e3 0c91 | 28fd 9f88 - 0x000000011984a584: 88a3 4e39 | 1f09 0071 | c001 0054 - - 0x000000011984a590: ; {external_word} - 0x000000011984a590: c8df f7b0 | 08a9 5339 | 6805 0035 | 9f4b 01f9 | 9f4f 01f9 | 826f 40f9 | 5f80 00f9 | bf03 0091 - 0x000000011984a5b0: fd7b c1a8 | 8807 40f9 | 4800 00b5 | c003 5fd6 - - 0x000000011984a5c0: ; {runtime_call StubRoutines (1)} - 0x000000011984a5c0: b006 bd17 | a083 1ff8 - - 0x000000011984a5c8: ; {runtime_call SharedRuntime::reguard_yellow_pages()} - 0x000000011984a5c8: 8864 87d2 | 6820 a1f2 | 2800 c0f2 | 0001 3fd6 | a083 5ff8 | edff ff17 | a083 1ff8 | e003 1caa - 0x000000011984a5e8: ; {runtime_call JavaThread::check_special_condition_for_native_trans(JavaThread*)} - 0x000000011984a5e8: 0836 8ad2 | 2823 a1f2 | 2800 c0f2 | 0001 3fd6 | a083 5ff8 | dfff ff17 | e10b bda9 | e313 01a9 - 0x000000011984a608: e51b 02a9 - - 0x000000011984a60c: ; {metadata({method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native')} - 0x000000011984a60c: 01bf 98d2 | c1bf a6f2 | 2100 c0f2 | e003 1caa | e833 bfa9 | 082e 83d2 | 6820 a1f2 | 2800 c0f2 - 0x000000011984a62c: 0001 3fd6 | e833 c1a8 | e313 41a9 | e51b 42a9 | e10b c3a8 | bdff ff17 | a083 1ff8 - - 0x000000011984a648: ; {metadata({method} {0x0000000135fec5f8} 'executePlan' '(IIJ[J[J)J' in 'org/apache/comet/Native')} - 0x000000011984a648: 01bf 98d2 | c1bf a6f2 | 2100 c0f2 | e003 1caa | e833 bfa9 | 0842 83d2 | 6820 a1f2 | 2800 c0f2 - 0x000000011984a668: 0001 3fd6 | e833 c1a8 | a083 5ff8 | caff ff17 -[/MachCode] - - -Compiled method (c1) 44945 18191 3 org.apache.comet.vector.NativeUtil::allocateArrowStructs (69 bytes) - total in heap [0x0000000112c2c890,0x0000000112c2dbc0] = 4912 - relocation [0x0000000112c2c9e8,0x0000000112c2cb50] = 360 - main code [0x0000000112c2cb80,0x0000000112c2d5c0] = 2624 - stub code [0x0000000112c2d5c0,0x0000000112c2d6f8] = 312 - oops [0x0000000112c2d6f8,0x0000000112c2d730] = 56 - metadata [0x0000000112c2d730,0x0000000112c2d7f8] = 200 - scopes data [0x0000000112c2d7f8,0x0000000112c2d9e8] = 496 - scopes pcs [0x0000000112c2d9e8,0x0000000112c2db88] = 416 - dependencies [0x0000000112c2db88,0x0000000112c2db90] = 8 - nul chk table [0x0000000112c2db90,0x0000000112c2dbc0] = 48 - -[Constant Pool (empty)] - -[MachCode] -[Entry Point] - # {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil' - # this: c_rarg1:c_rarg1 - = 'org/apache/comet/vector/NativeUtil' - # parm0: c_rarg2 = int - # [sp+0x120] (sp of caller) - 0x0000000112c2cb80: 2808 40b9 | 3f01 086b | c001 0054 - - 0x0000000112c2cb8c: ; {runtime_call ic_miss_stub} - 0x0000000112c2cb8c: 1d80 6e15 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 - 0x0000000112c2cbac: 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 -[Verified Entry Point] - 0x0000000112c2cbc0: 1f20 03d5 | e953 40d1 | 3f01 00f9 | ff83 04d1 | fd7b 11a9 | e163 00f9 - - 0x0000000112c2cbd8: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2cbd8: 00c3 90d2 | 60ca a6f2 | 2000 c0f2 | 03ac 40b9 | 6308 0011 | 03ac 00b9 | 6324 1f12 | 7f00 0071 - 0x0000000112c2cbf8: 203d 0054 | e2bb 00b9 - - 0x0000000112c2cc00: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2cc00: 00c3 90d2 | 60ca a6f2 | 2000 c0f2 | 0878 40f9 | 0805 0091 | 0878 00f9 - - 0x0000000112c2cc18: ; ImmutableOopMap {[192]=Oop } - ;*invokestatic $jacocoInit {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@0 - ; {static_call} - 0x0000000112c2cc18: 0a27 b595 | e073 00f9 | f3bb 40b9 - - 0x0000000112c2cc24: ; {metadata('org/apache/arrow/c/ArrowArray'[])} - 0x0000000112c2cc24: 0300 94d2 | 433c a0f2 | 0319 c0f2 | e403 00aa | 737e 40d3 | e503 13aa | e85f 40b2 | 7f02 08eb - 0x0000000112c2cc44: c23b 0054 | eb02 80d2 | 6b49 338b | 6bf1 7d92 | 8087 40f9 | 0b60 2b8b | 888f 40f9 | 7f01 08eb - 0x0000000112c2cc64: c83a 0054 | 8b87 00f9 | 6b01 00cb | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a4c 0129 | 6b41 00f1 - 0x0000000112c2cc84: c001 0054 | e803 0baa | 0a40 0091 | 0bfd 43d3 | 6821 00f1 | 4300 0054 - - 0x0000000112c2cc9c: ; {runtime_call StubRoutines (2)} - 0x0000000112c2cc9c: 892a 6e95 | 6b00 1036 | 5f7d 81a8 | 5f7d 81a8 | 4b00 0836 | 5f7d 81a8 | 4b00 0036 | 5f01 00f9 - 0x0000000112c2ccbc: bf3a 03d5 | e06b 00f9 | f3bb 40b9 - - 0x0000000112c2ccc8: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} - 0x0000000112c2ccc8: 038f 94d2 | 433c a0f2 | 0319 c0f2 | 737e 40d3 | e503 13aa | e85f 40b2 | 7f02 08eb | 0237 0054 - 0x0000000112c2cce8: eb02 80d2 | 6b49 338b | 6bf1 7d92 | 8087 40f9 | 0b60 2b8b | 888f 40f9 | 7f01 08eb | 0836 0054 - 0x0000000112c2cd08: 8b87 00f9 | 6b01 00cb | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a4c 0129 | 6b41 00f1 | c001 0054 - 0x0000000112c2cd28: e803 0baa | 0a40 0091 | 0bfd 43d3 | 6821 00f1 | 4300 0054 - - 0x0000000112c2cd3c: ; {runtime_call StubRoutines (2)} - 0x0000000112c2cd3c: 612a 6e95 | 6b00 1036 | 5f7d 81a8 | 5f7d 81a8 | 4b00 0836 | 5f7d 81a8 | 4b00 0036 | 5f01 00f9 - 0x0000000112c2cd5c: bf3a 03d5 | e06f 00f9 | e203 0032 - - 0x0000000112c2cd68: ; implicit exception: dispatches to 0x0000000112c2d3cc - 0x0000000112c2cd68: 810c 40b9 | 3f14 0071 | 0933 0054 | 8254 0039 - - 0x0000000112c2cd78: ; {oop(a 'java/lang/Class'{0x00000007039c8700} = 'scala/runtime/RichInt$')} - 0x0000000112c2cd78: 02e0 90d2 | 8273 a0f2 | e200 c0f2 | 4370 40b9 | 63f0 7dd3 - - 0x0000000112c2cd8c: ; {oop(a 'java/lang/Class'{0x0000000702fdfb20} = 'scala/Predef$')} - 0x0000000112c2cd8c: 0264 9fd2 | a25f a0f2 | e200 c0f2 | 4170 40b9 | 21f0 7dd3 - - 0x0000000112c2cda0: ; implicit exception: dispatches to 0x0000000112c2d3e8 - 0x0000000112c2cda0: 3f00 40f9 | e203 01aa - - 0x0000000112c2cda8: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2cda8: 05c3 90d2 | 65ca a6f2 | 2500 c0f2 | 4208 40b9 | 0219 c0f2 | a940 0491 | 2801 40f9 | 5f00 08eb - 0x0000000112c2cdc8: a100 0054 | a88c 40f9 | 0805 0091 | a88c 00f9 | 1c00 0014 | a980 0491 | 2801 40f9 | 5f00 08eb - 0x0000000112c2cde8: a100 0054 | a894 40f9 | 0805 0091 | a894 00f9 | 1400 0014 | a940 0491 | 2801 40f9 | c800 00b5 - 0x0000000112c2ce08: 2201 00f9 | e803 40b2 | a960 0491 | 2801 00f9 | 0c00 0014 | a980 0491 | 2801 40f9 | c800 00b5 - 0x0000000112c2ce28: 2201 00f9 | e803 40b2 | a9a0 0491 | 2801 00f9 | 0400 0014 | a880 40f9 | 0805 0091 | a880 00f9 - 0x0000000112c2ce48: 0200 a052 | e367 00f9 - - 0x0000000112c2ce50: ; ImmutableOopMap {[192]=Oop [200]=Oop [216]=Oop [208]=Oop [224]=Oop } - ;*invokevirtual intWrapper {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@27 (line 71) - ; {optimized virtual_call} - 0x0000000112c2ce50: 8010 b595 | e167 40f9 - - 0x0000000112c2ce58: ; implicit exception: dispatches to 0x0000000112c2d3ec - 0x0000000112c2ce58: 3f00 40f9 | e303 01aa - - 0x0000000112c2ce60: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2ce60: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 489c 40f9 | 0805 0091 | 489c 00f9 | e203 00aa | e3bb 40b9 - 0x0000000112c2ce80: ; ImmutableOopMap {[192]=Oop [216]=Oop [208]=Oop [224]=Oop } - ;*invokevirtual until$extension0 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@31 (line 71) - ; {optimized virtual_call} - 0x0000000112c2ce80: 44d0 b495 - - 0x0000000112c2ce84: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2ce84: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 4200 0691 | e163 40f9 | e303 01aa - - 0x0000000112c2ce9c: ; {metadata('org/apache/comet/vector/NativeUtil')} - 0x0000000112c2ce9c: 03f2 9ad2 | 433c a0f2 | 0319 c0f2 | 4904 40f9 | 6300 09ca | 68f4 7e92 | 4800 00b4 | 4304 00f9 - 0x0000000112c2cebc: e36b 40f9 | e403 03aa - - 0x0000000112c2cec4: ; {metadata('org/apache/arrow/c/ArrowArray'[])} - 0x0000000112c2cec4: 0400 94d2 | 443c a0f2 | 0419 c0f2 | 490c 40f9 | 8400 09ca | 88f4 7e92 | 0802 00b4 | e401 0837 - 0x0000000112c2cee4: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 8400 09ca | 490c 40f9 | 8400 09ca | 88f4 7e92 - 0x0000000112c2cf04: c800 00b4 | 490c 40f9 | 2901 7fb2 | 490c 00f9 | 0200 0014 | 440c 00f9 - - 0x0000000112c2cf1c: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2cf1c: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 48b8 40f9 | 0805 0091 | 48b8 00f9 - - 0x0000000112c2cf34: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c2cf34: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 44ac 40b9 | 8408 0011 | 44ac 00b9 | 844c 1f12 | 9f00 0071 - 0x0000000112c2cf54: e024 0054 - - 0x0000000112c2cf58: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2cf58: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 42c0 0891 - - 0x0000000112c2cf68: ; {oop(a 'java/lang/invoke/DirectMethodHandle$Constructor'{0x0000000712742d60})} - 0x0000000112c2cf68: 04ac 85d2 | 844e a2f2 | e400 c0f2 | e503 04aa | 8500 00b5 | 4900 40f9 | 2901 40b2 | 4900 00f9 - 0x0000000112c2cf88: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c2cf88: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 4894 40f9 | 0805 0091 | 4894 00f9 - - 0x0000000112c2cfa0: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2cfa0: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 45ac 40b9 | a508 0011 | 45ac 00b9 | a54c 1f12 | bf00 0071 - 0x0000000112c2cfc0: 8022 0054 - - 0x0000000112c2cfc4: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2cfc4: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0491 | e503 04aa | 8500 00b5 | 4904 40f9 | 2901 40b2 - 0x0000000112c2cfe4: 4904 00f9 - - 0x0000000112c2cfe8: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2cfe8: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4878 40f9 | 0805 0091 | 4878 00f9 - - 0x0000000112c2d000: ; {metadata(method data for {method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} - 0x0000000112c2d000: 0259 81d2 | c262 a5f2 | 2200 c0f2 | 45ac 40b9 | a508 0011 | 45ac 00b9 | a54c 1f12 | bf00 0071 - 0x0000000112c2d020: 8020 0054 - - 0x0000000112c2d024: ; {oop(a 'jdk/internal/misc/Unsafe'{0x0000000702af9ed0})} - 0x0000000112c2d024: 05da 93d2 | e555 a0f2 | e500 c0f2 | e203 05aa - - 0x0000000112c2d034: ; {metadata(method data for {method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} - 0x0000000112c2d034: 0659 81d2 | c662 a5f2 | 2600 c0f2 | c894 40f9 | 0805 0091 | c894 00f9 - - 0x0000000112c2d04c: ; {oop(a 'java/lang/Class'{0x0000000712742d88} = 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} - 0x0000000112c2d04c: 02b1 85d2 | 824e a2f2 | e200 c0f2 | e103 05aa | e483 0ea9 - - 0x0000000112c2d060: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [232]=Oop [208]=Oop [240]=Oop } - ;*invokevirtual allocateInstance {reexecute=0 rethrow=0 return_oop=0} - ; - java.lang.invoke.DirectMethodHandle::allocateInstance@12 (line 520) - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@1 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {optimized virtual_call} - 0x0000000112c2d060: 1c17 9295 - - 0x0000000112c2d064: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d064: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4240 0491 | e103 00aa | 8100 00b5 | 4900 40f9 | 2901 40b2 - 0x0000000112c2d084: 4900 00f9 - - 0x0000000112c2d088: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d088: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 42c0 0491 | e477 40f9 | e103 04aa | 8100 00b5 | 4904 40f9 - 0x0000000112c2d0a8: 2901 40b2 | 4904 00f9 - - 0x0000000112c2d0b0: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d0b0: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4890 40f9 | 0805 0091 | 4890 00f9 - - 0x0000000112c2d0c8: ; {metadata(method data for {method} {0x000000c8003a52f8} 'constructorMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} - 0x0000000112c2d0c8: 028d 81d2 | c262 a5f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 - 0x0000000112c2d0e8: 401b 0054 - - 0x0000000112c2d0ec: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d0ec: 0234 9dd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0591 - - 0x0000000112c2d0fc: ; {oop(a 'java/lang/invoke/MemberName'{0x0000000712742df8} = {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} - 0x0000000112c2d0fc: 01bf 85d2 | 814e a2f2 | e100 c0f2 | 8100 00b5 | 4900 40f9 | 2901 40b2 | 4900 00f9 - - 0x0000000112c2d118: ; implicit exception: dispatches to 0x0000000112c2d470 - 0x0000000112c2d118: 1f00 40f9 | e203 00aa - - 0x0000000112c2d120: ; {metadata(method data for {method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d120: 0134 9dd2 | 81a2 a5f2 | 2100 c0f2 | 28c4 40f9 | 0805 0091 | 28c4 00f9 - - 0x0000000112c2d138: ; {metadata(method data for {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} - 0x0000000112c2d138: 020b 91d2 | 62ca a6f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 - 0x0000000112c2d158: e018 0054 | e203 00aa - - 0x0000000112c2d160: ; {metadata(method data for {method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} - 0x0000000112c2d160: 010b 91d2 | 61ca a6f2 | 2100 c0f2 | 2878 40f9 | 0805 0091 | 2878 00f9 - - 0x0000000112c2d178: ; {metadata(method data for {method} {0x000000c800442c70} '' '()V' in 'java/lang/Object')} - 0x0000000112c2d178: 0224 95d2 | 0260 a5f2 | 2200 c0f2 | 41ac 40b9 | 2108 0011 | 41ac 00b9 | 214c 1f12 | 3f00 0071 - 0x0000000112c2d198: e017 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 | e163 40f9 | 28fc 43d3 | 080c 00b9 | 0200 01ca - 0x0000000112c2d1b8: 42fc 55d3 | 5f00 00f1 | 6118 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 | e36b 40f9 | 68fc 43d3 - 0x0000000112c2d1d8: 0810 00b9 | 0200 03ca | 42fc 55d3 | 5f00 00f1 | 6118 0054 | 82e3 4039 | 5f00 0071 | 8118 0054 - 0x0000000112c2d1f8: e46f 40f9 | 88fc 43d3 | 0814 00b9 | 0200 04ca | 42fc 55d3 | 5f00 00f1 | 6118 0054 | bf3a 03d5 - 0x0000000112c2d218: ; {metadata(method data for {method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c2d218: 02f1 9cd2 | 82a2 a5f2 | 2200 c0f2 | 4200 0691 | e103 00aa | 8100 00b5 | 4900 40f9 | 2901 40b2 - 0x0000000112c2d238: 4900 00f9 - - 0x0000000112c2d23c: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2d23c: 02c3 90d2 | 62ca a6f2 | 2200 c0f2 | 4280 0691 | e103 00aa | a100 00b5 | 4900 40f9 | 2901 40b2 - 0x0000000112c2d25c: 4900 00f9 | 1600 0014 | 2108 40b9 | 0119 c0f2 | 4900 40f9 | 2100 09ca | 28f4 7e92 | 0802 00b4 - 0x0000000112c2d27c: e101 0837 | a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 2100 09ca | 4900 40f9 | 2100 09ca - 0x0000000112c2d29c: 28f4 7e92 | c800 00b4 | 4900 40f9 | 2901 7fb2 | 4900 00f9 | 0200 0014 | 4100 00f9 | e17b 40f9 - 0x0000000112c2d2bc: ; implicit exception: dispatches to 0x0000000112c2d52c - 0x0000000112c2d2bc: 3f00 40f9 | e203 01aa - - 0x0000000112c2d2c4: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2d2c4: 05c3 90d2 | 65ca a6f2 | 2500 c0f2 | a8d8 40f9 | 0805 0091 | a8d8 00f9 | e203 00aa - - 0x0000000112c2d2e0: ; ImmutableOopMap {[224]=Oop [208]=Oop [216]=Oop } - ;*invokevirtual foreach$mVc$sp {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@44 (line 71) - ; {optimized virtual_call} - 0x0000000112c2d2e0: 2c28 b495 | e303 0032 | e073 40f9 | 040c 40b9 | 9f18 0071 | e911 0054 | 0358 0039 - - 0x0000000112c2d2fc: ; {metadata('scala/Tuple2')} - 0x0000000112c2d2fc: 0356 92d2 | 8321 a0f2 | 0319 c0f2 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb | 8811 0054 - 0x0000000112c2d31c: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 | bf3a 03d5 - 0x0000000112c2d33c: e203 00aa - - 0x0000000112c2d340: ; {metadata(method data for {method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2d340: 03c3 90d2 | 63ca a6f2 | 2300 c0f2 | 68f4 40f9 | 0805 0091 | 68f4 00f9 | e20f 4da9 | e103 00aa - 0x0000000112c2d360: e07f 00f9 - - 0x0000000112c2d364: ; ImmutableOopMap {[248]=Oop [224]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@60 (line 78) - ; {optimized virtual_call} - 0x0000000112c2d364: bb23 9995 | e003 0032 | e173 40f9 | 220c 40b9 | 5f1c 0071 | c90e 0054 | 205c 0039 | e07f 40f9 - 0x0000000112c2d384: fd7b 51a9 | ff83 0491 - - 0x0000000112c2d38c: ; {poll_return} - 0x0000000112c2d38c: 88a3 41f9 | ff63 28eb | a80e 0054 | c003 5fd6 - - 0x0000000112c2d39c: ; {metadata({method} {0x0000000135fed5f8} 'allocateArrowStructs' '(I)Lscala/Tuple2;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2d39c: 08bf 9ad2 | c8bf a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d3b4: ; ImmutableOopMap {c_rarg1=Oop [192]=Oop } - ;*synchronization entry - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@-1 - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d3b4: 5352 7195 | 11fe ff17 - - 0x0000000112c2d3bc: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop } - ;*anewarray {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@5 (line 68) - ; {runtime_call new_object_array Runtime1 stub} - 0x0000000112c2d3bc: 5143 7195 | 40fe ff17 - - 0x0000000112c2d3c4: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop } - ;*anewarray {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@11 (line 69) - ; {runtime_call new_object_array Runtime1 stub} - 0x0000000112c2d3c4: 4f43 7195 | 66fe ff17 - - 0x0000000112c2d3cc: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@19 (line 69) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c2d3cc: cd3e 7195 | a800 80d2 | e903 04aa - - 0x0000000112c2d3d8: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c2d3d8: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c2d3e4: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@19 (line 69) - 0x0000000112c2d3e4: c003 3fd6 - - 0x0000000112c2d3e8: ; ImmutableOopMap {[192]=Oop c_rarg4=Oop [224]=Oop [208]=Oop c_rarg0=Oop [216]=Oop c_rarg3=Oop c_rarg1=Oop } - ;*invokevirtual intWrapper {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@27 (line 71) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c2d3e8: c63e 7195 - - 0x0000000112c2d3ec: ; ImmutableOopMap {[192]=Oop [216]=Oop [208]=Oop [224]=Oop c_rarg1=Oop } - ;*invokevirtual until$extension0 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@31 (line 71) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c2d3ec: c53e 7195 - - 0x0000000112c2d3f0: ; {metadata({method} {0x000000c8000d7fc0} 'linkToTargetMethod' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c2d3f0: 08f8 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d408: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop } - ;*synchronization entry - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d408: 3e52 7195 | d3fe ff17 - - 0x0000000112c2d410: ; {metadata({method} {0x000000c8000d75a0} 'newInvokeSpecial' '(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle$Holder')} - 0x0000000112c2d410: 08b4 8ed2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d428: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop } - ;*synchronization entry - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@-1 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d428: 3652 7195 | e6fe ff17 - - 0x0000000112c2d430: ; {metadata({method} {0x000000c8003a4850} 'allocateInstance' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} - 0x0000000112c2d430: 080a 89d2 | 4807 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d448: ; ImmutableOopMap {[216]=Oop [224]=Oop c_rarg0=Oop c_rarg1=Oop [192]=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop } - ;*synchronization entry - ; - java.lang.invoke.DirectMethodHandle::allocateInstance@-1 (line 519) - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@1 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d448: 2e52 7195 | f6fe ff17 - - 0x0000000112c2d450: ; {metadata({method} {0x000000c8003a52f8} 'constructorMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/DirectMethodHandle')} - 0x0000000112c2d450: 085f 8ad2 | 4807 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d468: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } - ;*synchronization entry - ; - java.lang.invoke.DirectMethodHandle::constructorMethod@-1 (line 513) - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@7 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d468: 2652 7195 | 20ff ff17 - - 0x0000000112c2d470: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } - ;*invokestatic linkToSpecial {reexecute=0 rethrow=0 return_oop=0} - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c2d470: a43e 7195 - - 0x0000000112c2d474: ; {metadata({method} {0x00000001360517d0} '' '(Lorg/apache/comet/vector/NativeUtil;[Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)V' in 'org/apache/comet/vector/NativeUtil$$Lambda$3988+0x000000c801e28000')} - 0x0000000112c2d474: 08fa 82d2 | a8c0 a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d48c: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } - ;*synchronization entry - ; - org.apache.comet.vector.NativeUtil$$Lambda$3988/0x000000c801e28000::@-1 - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d48c: 1d52 7195 | 33ff ff17 - - 0x0000000112c2d494: ; {metadata({method} {0x000000c800442c70} '' '()V' in 'java/lang/Object')} - 0x0000000112c2d494: 088e 85d2 | 8808 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c2d4ac: ; ImmutableOopMap {[216]=Oop [224]=Oop [192]=Oop [208]=Oop [240]=Oop c_rarg0=Oop } - ;*synchronization entry - ; - java.lang.Object::@-1 (line 44) - ; - org.apache.comet.vector.NativeUtil$$Lambda$3988/0x000000c801e28000::@1 - ; - java.lang.invoke.DirectMethodHandle$Holder::newInvokeSpecial@22 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@7 - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@39 (line 71) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c2d4ac: 1552 7195 | 3bff ff17 | 020c 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 - - 0x0000000112c2d4c4: ; {runtime_call g1_pre_barrier_slow} - 0x0000000112c2d4c4: 8f53 7195 | 38ff ff17 | c1e7 ffb4 | e003 00f9 - - 0x0000000112c2d4d4: ; {runtime_call g1_post_barrier_slow} - 0x0000000112c2d4d4: 4b54 7195 | 3bff ff17 | 0210 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 - - 0x0000000112c2d4ec: ; {runtime_call g1_pre_barrier_slow} - 0x0000000112c2d4ec: 8553 7195 | 38ff ff17 | c3e7 ffb4 | e003 00f9 - - 0x0000000112c2d4fc: ; {runtime_call g1_post_barrier_slow} - 0x0000000112c2d4fc: 4154 7195 | 3bff ff17 | 0214 40b9 | 42f0 7dd3 | 62e7 ffb4 | e203 00f9 - - 0x0000000112c2d514: ; {runtime_call g1_pre_barrier_slow} - 0x0000000112c2d514: 7b53 7195 | 38ff ff17 | c4e7 ffb4 | e003 00f9 - - 0x0000000112c2d524: ; {runtime_call g1_post_barrier_slow} - 0x0000000112c2d524: 3754 7195 | 3bff ff17 - - 0x0000000112c2d52c: ; ImmutableOopMap {[224]=Oop c_rarg0=Oop c_rarg3=Oop [208]=Oop c_rarg4=Oop [216]=Oop c_rarg1=Oop } - ;*invokevirtual foreach$mVc$sp {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@44 (line 71) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c2d52c: 753e 7195 | e807 7fb2 | e903 00aa - - 0x0000000112c2d538: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c2d538: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c2d544: ; ImmutableOopMap {[208]=Oop [216]=Oop c_rarg0=Oop [224]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@51 (line 71) - 0x0000000112c2d544: c003 3fd6 - - 0x0000000112c2d548: ; ImmutableOopMap {[208]=Oop [216]=Oop [224]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@52 (line 78) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c2d548: ae40 7195 | 7cff ff17 | e80b 40b2 | e903 01aa - - 0x0000000112c2d558: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c2d558: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c2d564: ; ImmutableOopMap {[248]=Oop c_rarg1=Oop [224]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::allocateArrowStructs@67 (line 78) - 0x0000000112c2d564: c003 3fd6 - - 0x0000000112c2d568: ; {internal_word} - 0x0000000112c2d568: 28f1 ff10 | 88af 01f9 - - 0x0000000112c2d570: ; {runtime_call SafepointBlob} - 0x0000000112c2d570: 2497 6e15 | 1f20 03d5 | 1f20 03d5 | 80eb 41f9 | 9feb 01f9 | 9fef 01f9 | fd7b 51a9 | ff83 0491 - 0x0000000112c2d590: ; {runtime_call unwind_exception Runtime1 stub} - 0x0000000112c2d590: 5c02 6f15 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 - 0x0000000112c2d5b0: 0000 0000 | 0000 0000 | 0000 0000 | 0000 0000 -[Stub Code] - 0x0000000112c2d5c0: ; {no_reloc} - 0x0000000112c2d5c0: df3f 03d5 - - 0x0000000112c2d5c4: ; {metadata(NULL)} - 0x0000000112c2d5c4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d5e0: ; {trampoline_stub} - 0x0000000112c2d5e0: 4800 0058 | 0001 1fd6 | 4068 9719 | 0100 0000 - - 0x0000000112c2d5f0: ; {static_stub} - 0x0000000112c2d5f0: df3f 03d5 - - 0x0000000112c2d5f4: ; {metadata(NULL)} - 0x0000000112c2d5f4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d610: ; {trampoline_stub} - 0x0000000112c2d610: 4800 0058 | 0001 1fd6 | 5010 9719 | 0100 0000 - - 0x0000000112c2d620: ; {static_stub} - 0x0000000112c2d620: df3f 03d5 - - 0x0000000112c2d624: ; {metadata(NULL)} - 0x0000000112c2d624: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d640: ; {trampoline_stub} - 0x0000000112c2d640: 4800 0058 | 0001 1fd6 | 900f 9619 | 0100 0000 - - 0x0000000112c2d650: ; {static_stub} - 0x0000000112c2d650: df3f 03d5 - - 0x0000000112c2d654: ; {metadata(NULL)} - 0x0000000112c2d654: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d670: ; {trampoline_stub} - 0x0000000112c2d670: 4800 0058 | 0001 1fd6 | d02c 0b19 | 0100 0000 - - 0x0000000112c2d680: ; {static_stub} - 0x0000000112c2d680: df3f 03d5 - - 0x0000000112c2d684: ; {metadata(NULL)} - 0x0000000112c2d684: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d6a0: ; {trampoline_stub} - 0x0000000112c2d6a0: 4800 0058 | 0001 1fd6 | 9073 9319 | 0100 0000 - - 0x0000000112c2d6b0: ; {static_stub} - 0x0000000112c2d6b0: df3f 03d5 - - 0x0000000112c2d6b4: ; {metadata(NULL)} - 0x0000000112c2d6b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c2d6d0: ; {trampoline_stub} - 0x0000000112c2d6d0: 4800 0058 | 0001 1fd6 | 5062 2719 | 0100 0000 -[Exception Handler] - 0x0000000112c2d6e0: ; {runtime_call handle_exception_from_callee Runtime1 stub} - 0x0000000112c2d6e0: 8845 7195 | c1d5 bbd4 | 3127 2a09 | 0100 0000 -[Deopt Handler Code] - 0x0000000112c2d6f0: 1e00 0010 - - 0x0000000112c2d6f4: ; {runtime_call DeoptimizationBlob} - 0x0000000112c2d6f4: 9397 6e15 -[/MachCode] - - -Compiled method (c1) 44954 18196 3 org.apache.comet.vector.NativeUtil::getNextBatch (296 bytes) - total in heap [0x0000000112c2f890,0x0000000112c32608] = 11640 - relocation [0x0000000112c2f9e8,0x0000000112c2fda8] = 960 - main code [0x0000000112c2fdc0,0x0000000112c31580] = 6080 - stub code [0x0000000112c31580,0x0000000112c31a48] = 1224 - oops [0x0000000112c31a48,0x0000000112c31aa0] = 88 - metadata [0x0000000112c31aa0,0x0000000112c31bf0] = 336 - scopes data [0x0000000112c31bf0,0x0000000112c32170] = 1408 - scopes pcs [0x0000000112c32170,0x0000000112c32590] = 1056 - dependencies [0x0000000112c32590,0x0000000112c32598] = 8 - nul chk table [0x0000000112c32598,0x0000000112c32608] = 112 - -[Constant Pool (empty)] - -[MachCode] -[Entry Point] - # {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil' - # this: c_rarg1:c_rarg1 - = 'org/apache/comet/vector/NativeUtil' - # parm0: c_rarg2 = int - # parm1: c_rarg3:c_rarg3 - = 'scala/Function2' - # [sp+0x150] (sp of caller) - 0x0000000112c2fdc0: 2808 40b9 | 3f01 086b | c001 0054 - - 0x0000000112c2fdcc: ; {runtime_call ic_miss_stub} - 0x0000000112c2fdcc: 8d73 6e15 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 - 0x0000000112c2fdec: 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 | 1f20 03d5 -[Verified Entry Point] - 0x0000000112c2fe00: 1f20 03d5 | e953 40d1 | 3f01 00f9 | ff43 05d1 | fd7b 14a9 | e347 00f9 - - 0x0000000112c2fe18: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fe18: 00b9 91d2 | 60ca a6f2 | 2000 c0f2 | 04ac 40b9 | 8408 0011 | 04ac 00b9 | 8424 1f12 | 9f00 0071 - 0x0000000112c2fe38: 60a9 0054 | e283 00b9 | e13f 00f9 - - 0x0000000112c2fe44: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fe44: 00b9 91d2 | 60ca a6f2 | 2000 c0f2 | 0878 40f9 | 0805 0091 | 0878 00f9 - - 0x0000000112c2fe5c: ; ImmutableOopMap {[120]=Oop [136]=Oop } - ;*invokestatic $jacocoInit {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@0 - ; {static_call} - 0x0000000112c2fe5c: 791a b595 | e04b 00f9 | e13f 40f9 - - 0x0000000112c2fe68: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fe68: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 - - 0x0000000112c2fe74: ; {metadata('org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fe74: 08f2 9ad2 | 483c a0f2 | 0819 c0f2 | 4940 0491 | 2801 00f9 | 488c 40f9 | 0805 0091 | 488c 00f9 - 0x0000000112c2fe94: e283 40b9 | e13f 40f9 - - 0x0000000112c2fe9c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop } - ;*invokevirtual allocateArrowStructs {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@7 (line 227) - ; {optimized virtual_call} - 0x0000000112c2fe9c: 49f3 ff97 | e103 1faa | 1f00 01eb - - 0x0000000112c2fea8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fea8: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 0827 80d2 | 0929 80d2 | 0201 899a | 2368 62f8 | 6304 0091 - 0x0000000112c2fec8: 2368 22f8 | 209f 0054 - - 0x0000000112c2fed0: ; implicit exception: dispatches to 0x0000000112c31384 - 0x0000000112c2fed0: 1f00 40f9 | e103 00aa - - 0x0000000112c2fed8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2fed8: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 49a0 0591 | 2801 40f9 | 3f00 08eb - 0x0000000112c2fef8: a100 0054 | 48b8 40f9 | 0805 0091 | 48b8 00f9 | 1c00 0014 | 49e0 0591 | 2801 40f9 | 3f00 08eb - 0x0000000112c2ff18: a100 0054 | 48c0 40f9 | 0805 0091 | 48c0 00f9 | 1400 0014 | 49a0 0591 | 2801 40f9 | c800 00b5 - 0x0000000112c2ff38: 2101 00f9 | e803 40b2 | 49c0 0591 | 2801 00f9 | 0c00 0014 | 49e0 0591 | 2801 40f9 | c800 00b5 - 0x0000000112c2ff58: 2101 00f9 | e803 40b2 | 4900 0691 | 2801 00f9 | 0400 0014 | 48ac 40f9 | 0805 0091 | 48ac 00f9 - 0x0000000112c2ff78: e103 00aa | e04f 00f9 | 0956 92d2 | 8921 a0f2 | 0919 c0f2 - - 0x0000000112c2ff8c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [152]=Oop } - ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@19 (line 227) - ; {virtual_call} - 0x0000000112c2ff8c: cd63 8d95 | 0001 00b5 - - 0x0000000112c2ff94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2ff94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6824 4639 | 0801 40b2 | 6824 0639 | 3700 0014 - - 0x0000000112c2ffb0: ; {metadata('org/apache/arrow/c/ArrowArray'[])} - 0x0000000112c2ffb0: 0200 94d2 | 423c a0f2 | 0219 c0f2 | 0308 40b9 | 0319 c0f2 | 6820 40f9 | 5f00 08eb | e104 0054 - 0x0000000112c2ffd0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c2ffd0: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0208 40b9 | 0219 c0f2 | 6980 0691 | 2801 40f9 | 5f00 08eb - 0x0000000112c2fff0: a100 0054 | 68d4 40f9 | 0805 0091 | 68d4 00f9 | 2200 0014 | 69c0 0691 | 2801 40f9 | 5f00 08eb - 0x0000000112c30010: a100 0054 | 68dc 40f9 | 0805 0091 | 68dc 00f9 | 1a00 0014 | 6980 0691 | 2801 40f9 | c800 00b5 - 0x0000000112c30030: 2201 00f9 | e803 40b2 | 69a0 0691 | 2801 00f9 | 1200 0014 | 69c0 0691 | 2801 40f9 | c800 00b5 - 0x0000000112c30050: 2201 00f9 | e803 40b2 | 69e0 0691 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c30068: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30068: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68c8 40f9 | 0805 00d1 | 68c8 00f9 | c204 0014 | 0100 0014 - 0x0000000112c30088: e14f 40f9 - - 0x0000000112c3008c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3008c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 4960 0791 | 2801 40f9 | 3f00 08eb - 0x0000000112c300ac: a100 0054 | 48f0 40f9 | 0805 0091 | 48f0 00f9 | 1c00 0014 | 49a0 0791 | 2801 40f9 | 3f00 08eb - 0x0000000112c300cc: a100 0054 | 48f8 40f9 | 0805 0091 | 48f8 00f9 | 1400 0014 | 4960 0791 | 2801 40f9 | c800 00b5 - 0x0000000112c300ec: 2101 00f9 | e803 40b2 | 4980 0791 | 2801 00f9 | 0c00 0014 | 49a0 0791 | 2801 40f9 | c800 00b5 - 0x0000000112c3010c: 2101 00f9 | e803 40b2 | 49c0 0791 | 2801 00f9 | 0400 0014 | 48e4 40f9 | 0805 0091 | 48e4 00f9 - 0x0000000112c3012c: e14f 40f9 | e053 00f9 | 0956 92d2 | 8921 a0f2 | 0919 c0f2 - - 0x0000000112c30140: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop } - ;*invokevirtual _2 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@29 (line 227) - ; {virtual_call} - 0x0000000112c30140: e02c 9995 | 0001 00b5 - - 0x0000000112c30148: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30148: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88e4 4739 | 0801 40b2 | 88e4 0739 | 3700 0014 - - 0x0000000112c30164: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} - 0x0000000112c30164: 018f 94d2 | 413c a0f2 | 0119 c0f2 | 0408 40b9 | 0419 c0f2 | 8820 40f9 | 3f00 08eb | e104 0054 - 0x0000000112c30184: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30184: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0108 40b9 | 0119 c0f2 | 8940 0891 | 2801 40f9 | 3f00 08eb - 0x0000000112c301a4: a100 0054 | 880c 41f9 | 0805 0091 | 880c 01f9 | 2200 0014 | 8980 0891 | 2801 40f9 | 3f00 08eb - 0x0000000112c301c4: a100 0054 | 8814 41f9 | 0805 0091 | 8814 01f9 | 1a00 0014 | 8940 0891 | 2801 40f9 | c800 00b5 - 0x0000000112c301e4: 2101 00f9 | e803 40b2 | 8960 0891 | 2801 00f9 | 1200 0014 | 8980 0891 | 2801 40f9 | c800 00b5 - 0x0000000112c30204: 2101 00f9 | e803 40b2 | 89a0 0891 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c3021c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3021c: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8800 41f9 | 0805 00d1 | 8800 01f9 | 5704 0014 | 0100 0014 - 0x0000000112c3023c: e103 00aa - - 0x0000000112c30240: ; {metadata('scala/Tuple2')} - 0x0000000112c30240: 0356 92d2 | 8321 a0f2 | 0319 c0f2 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb | e889 0054 - 0x0000000112c30260: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 | bf3a 03d5 - 0x0000000112c30280: e057 00f9 | e203 00aa - - 0x0000000112c30288: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30288: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 681c 41f9 | 0805 0091 | 681c 01f9 | e253 40f9 | e303 01aa - 0x0000000112c302a8: e103 00aa - - 0x0000000112c302ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@45 (line 227) - ; {optimized virtual_call} - 0x0000000112c302ac: e917 9995 - - 0x0000000112c302b0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c302b0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2024 41f9 | 0004 0091 | 2024 01f9 | e057 40f9 | e103 00aa - 0x0000000112c302d0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c302d0: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 2108 40b9 | 0119 c0f2 | 8960 0a91 | 2801 40f9 | 3f00 08eb - 0x0000000112c302f0: a100 0054 | 8850 41f9 | 0805 0091 | 8850 01f9 | 1c00 0014 | 89a0 0a91 | 2801 40f9 | 3f00 08eb - 0x0000000112c30310: a100 0054 | 8858 41f9 | 0805 0091 | 8858 01f9 | 1400 0014 | 8960 0a91 | 2801 40f9 | c800 00b5 - 0x0000000112c30330: 2101 00f9 | e803 40b2 | 8980 0a91 | 2801 00f9 | 0c00 0014 | 89a0 0a91 | 2801 40f9 | c800 00b5 - 0x0000000112c30350: 2101 00f9 | e803 40b2 | 89c0 0a91 | 2801 00f9 | 0400 0014 | 8844 41f9 | 0805 0091 | 8844 01f9 - 0x0000000112c30370: e103 00aa - - 0x0000000112c30374: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } - ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@74 (line 227) - ; {optimized virtual_call} - 0x0000000112c30374: d762 8d95 | 0001 00b5 - - 0x0000000112c3037c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3037c: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e4 4a39 | 0801 40b2 | 68e4 0a39 | 3700 0014 - - 0x0000000112c30398: ; {metadata('org/apache/arrow/c/ArrowArray'[])} - 0x0000000112c30398: 0200 94d2 | 423c a0f2 | 0219 c0f2 | 0308 40b9 | 0319 c0f2 | 6820 40f9 | 5f00 08eb | e104 0054 - 0x0000000112c303b8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c303b8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0208 40b9 | 0219 c0f2 | 6940 0b91 | 2801 40f9 | 5f00 08eb - 0x0000000112c303d8: a100 0054 | 686c 41f9 | 0805 0091 | 686c 01f9 | 2200 0014 | 6980 0b91 | 2801 40f9 | 5f00 08eb - 0x0000000112c303f8: a100 0054 | 6874 41f9 | 0805 0091 | 6874 01f9 | 1a00 0014 | 6940 0b91 | 2801 40f9 | c800 00b5 - 0x0000000112c30418: 2201 00f9 | e803 40b2 | 6960 0b91 | 2801 00f9 | 1200 0014 | 6980 0b91 | 2801 40f9 | c800 00b5 - 0x0000000112c30438: 2201 00f9 | e803 40b2 | 69a0 0b91 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c30450: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30450: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6860 41f9 | 0805 00d1 | 6860 01f9 | ce03 0014 | 0100 0014 - 0x0000000112c30470: e157 40f9 - - 0x0000000112c30474: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30474: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 2108 40b9 | 0119 c0f2 | 4920 0c91 | 2801 40f9 | 3f00 08eb - 0x0000000112c30494: a100 0054 | 4888 41f9 | 0805 0091 | 4888 01f9 | 1c00 0014 | 4960 0c91 | 2801 40f9 | 3f00 08eb - 0x0000000112c304b4: a100 0054 | 4890 41f9 | 0805 0091 | 4890 01f9 | 1400 0014 | 4920 0c91 | 2801 40f9 | c800 00b5 - 0x0000000112c304d4: 2101 00f9 | e803 40b2 | 4940 0c91 | 2801 00f9 | 0c00 0014 | 4960 0c91 | 2801 40f9 | c800 00b5 - 0x0000000112c304f4: 2101 00f9 | e803 40b2 | 4980 0c91 | 2801 00f9 | 0400 0014 | 487c 41f9 | 0805 0091 | 487c 01f9 - 0x0000000112c30514: e157 40f9 | e05b 00f9 - - 0x0000000112c3051c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [176]=Oop } - ;*invokevirtual _2 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@84 (line 227) - ; {optimized virtual_call} - 0x0000000112c3051c: ed2b 9995 | 0001 00b5 - - 0x0000000112c30524: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30524: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88a4 4c39 | 0801 40b2 | 88a4 0c39 | 3700 0014 - - 0x0000000112c30540: ; {metadata('org/apache/arrow/c/ArrowSchema'[])} - 0x0000000112c30540: 058f 94d2 | 453c a0f2 | 0519 c0f2 | 0408 40b9 | 0419 c0f2 | 8820 40f9 | bf00 08eb | e104 0054 - 0x0000000112c30560: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30560: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0508 40b9 | 0519 c0f2 | 8900 0d91 | 2801 40f9 | bf00 08eb - 0x0000000112c30580: a100 0054 | 88a4 41f9 | 0805 0091 | 88a4 01f9 | 2200 0014 | 8940 0d91 | 2801 40f9 | bf00 08eb - 0x0000000112c305a0: a100 0054 | 88ac 41f9 | 0805 0091 | 88ac 01f9 | 1a00 0014 | 8900 0d91 | 2801 40f9 | c800 00b5 - 0x0000000112c305c0: 2501 00f9 | e803 40b2 | 8920 0d91 | 2801 00f9 | 1200 0014 | 8940 0d91 | 2801 40f9 | c800 00b5 - 0x0000000112c305e0: 2501 00f9 | e803 40b2 | 8960 0d91 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c305f8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c305f8: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8898 41f9 | 0805 00d1 | 8898 01f9 | 6603 0014 | 0100 0014 - 0x0000000112c30618: e503 00aa | e303 0032 | e04b 40f9 - - 0x0000000112c30624: ; implicit exception: dispatches to 0x0000000112c313b0 - 0x0000000112c30624: 040c 40b9 | 9f7c 0071 | 496c 0054 | 03bc 0039 - - 0x0000000112c30634: ; {metadata('scala/collection/mutable/ArrayOps$ofRef')} - 0x0000000112c30634: 0303 94d2 | 4320 a0f2 | 0319 c0f2 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb | e86b 0054 - 0x0000000112c30654: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 - - 0x0000000112c3066c: ; {oop(a 'java/lang/Class'{0x0000000702fdfb20} = 'scala/Predef$')} - 0x0000000112c3066c: 0364 9fd2 | a35f a0f2 | e300 c0f2 | 6170 40b9 | 21f0 7dd3 - - 0x0000000112c30680: ; implicit exception: dispatches to 0x0000000112c313d4 - 0x0000000112c30680: 3f00 40f9 | e203 01aa - - 0x0000000112c30688: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30688: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88d0 41f9 | 0805 0091 | 88d0 01f9 | e25b 40f9 | e317 0ca9 - 0x0000000112c306a8: e05f 00f9 - - 0x0000000112c306ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } - ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@110 (line 229) - ; {optimized virtual_call} - 0x0000000112c306ac: a9f2 9a95 | e25f 40f9 - - 0x0000000112c306b4: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c306b4: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28ec 41f9 | 0805 0091 | 28ec 01f9 | e203 00aa | e15f 40f9 - 0x0000000112c306d4: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@113 (line 229) - ; {optimized virtual_call} - 0x0000000112c306d4: 6fe0 9b95 - - 0x0000000112c306d8: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c306d8: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2120 0791 - - 0x0000000112c306e8: ; {oop(a 'java/lang/invoke/BoundMethodHandle$Species_L'{0x000000071276b5a0})} - 0x0000000112c306e8: 00b4 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c30708: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30708: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28f4 41f9 | 0805 0091 | 28f4 01f9 - - 0x0000000112c30720: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30720: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 22ac 40b9 | 4208 0011 | 22ac 00b9 | 424c 1f12 | 5f00 0071 - 0x0000000112c30740: c064 0054 - - 0x0000000112c30744: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c30744: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 2100 0691 | 8000 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c30764: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30764: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2894 40f9 | 0805 0091 | 2894 00f9 - - 0x0000000112c3077c: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c3077c: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 20ac 40b9 | 0008 0011 | 20ac 00b9 | 004c 1f12 | 1f00 0071 - 0x0000000112c3079c: e062 0054 - - 0x0000000112c307a0: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c307a0: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2180 0591 - - 0x0000000112c307b0: ; {oop(a 'org/apache/comet/vector/NativeUtil$$Lambda$3991+0x000000c801e22000'{0x000000071276b5c0})} - 0x0000000112c307b0: 00b8 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c307d0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c307d0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 21e0 0f91 | e203 00aa | a200 00b5 | 2900 40f9 | 2901 40b2 - 0x0000000112c307f0: 2900 00f9 | 1700 0014 - - 0x0000000112c307f8: ; {metadata('org/apache/comet/vector/NativeUtil$$Lambda$3991+0x000000c801e22000')} - 0x0000000112c307f8: 0200 84d2 | 423c a0f2 | 0219 c0f2 | 2900 40f9 | 4200 09ca | 48f4 7e92 | 0802 00b4 | e201 0837 - 0x0000000112c30818: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 4200 09ca | 2900 40f9 | 4200 09ca | 48f4 7e92 - 0x0000000112c30838: c800 00b4 | 2900 40f9 | 2901 7fb2 | 2900 00f9 | 0200 0014 | 2200 00f9 - - 0x0000000112c30850: ; {oop(a 'java/lang/Class'{0x00000007039c32f8} = 'scala/Array$')} - 0x0000000112c30850: 025f 86d2 | 8273 a0f2 | e200 c0f2 | 4370 40b9 | 63f0 7dd3 - - 0x0000000112c30864: ; {oop(a 'java/lang/Class'{0x00000007039c31e8} = 'scala/reflect/ClassTag$')} - 0x0000000112c30864: 043d 86d2 | 8473 a0f2 | e400 c0f2 | 8170 40b9 | 21f0 7dd3 - - 0x0000000112c30878: ; implicit exception: dispatches to 0x0000000112c31418 - 0x0000000112c30878: 3f00 40f9 | e503 01aa - - 0x0000000112c30880: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30880: 06b9 91d2 | 66ca a6f2 | 2600 c0f2 | c804 42f9 | 0805 0091 | c804 02f9 | e213 0ea9 | e303 0da9 - 0x0000000112c308a0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [208]=Oop [224]=Oop [216]=Oop [232]=Oop } - ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@127 (line 229) - ; {optimized virtual_call} - 0x0000000112c308a0: ecd7 c595 | e36b 40f9 - - 0x0000000112c308a8: ; implicit exception: dispatches to 0x0000000112c3141c - 0x0000000112c308a8: 7f00 40f9 | e203 03aa - - 0x0000000112c308b0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c308b0: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2820 42f9 | 0805 0091 | 2820 02f9 | e203 00aa | e103 03aa - 0x0000000112c308d0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [224]=Oop [216]=Oop [232]=Oop } - ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@130 (line 229) - ; {optimized virtual_call} - 0x0000000112c308d0: a020 bf95 | e25f 40f9 - - 0x0000000112c308d8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c308d8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 683c 42f9 | 0805 0091 | 683c 02f9 | e26f 40f9 | e303 00aa - 0x0000000112c308f8: e15f 40f9 - - 0x0000000112c308fc: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop } - ;*invokevirtual map {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@133 (line 229) - ; {optimized virtual_call} - 0x0000000112c308fc: 6523 bf95 | 0001 00b5 - - 0x0000000112c30904: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30904: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88a4 5239 | 0801 40b2 | 88a4 1239 | 3700 0014 - - 0x0000000112c30920: ; {metadata({type array long})} - 0x0000000112c30920: 0530 82d2 | 0500 a0f2 | 0519 c0f2 | 0408 40b9 | 0419 c0f2 | 881c 40f9 | bf00 08eb | e104 0054 - 0x0000000112c30940: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30940: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 0508 40b9 | 0519 c0f2 | 8900 1391 | 2801 40f9 | bf00 08eb - 0x0000000112c30960: a100 0054 | 8864 42f9 | 0805 0091 | 8864 02f9 | 2200 0014 | 8940 1391 | 2801 40f9 | bf00 08eb - 0x0000000112c30980: a100 0054 | 886c 42f9 | 0805 0091 | 886c 02f9 | 1a00 0014 | 8900 1391 | 2801 40f9 | c800 00b5 - 0x0000000112c309a0: 2501 00f9 | e803 40b2 | 8920 1391 | 2801 00f9 | 1200 0014 | 8940 1391 | 2801 40f9 | c800 00b5 - 0x0000000112c309c0: 2501 00f9 | e803 40b2 | 8960 1391 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c309d8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c309d8: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 8858 42f9 | 0805 00d1 | 8858 02f9 | 8c02 0014 | 0100 0014 - 0x0000000112c309f8: e503 00aa | e303 0032 | e04b 40f9 | 040c 40b9 | 9f80 0071 | e950 0054 | 03c0 0039 - - 0x0000000112c30a14: ; {metadata('scala/collection/mutable/ArrayOps$ofRef')} - 0x0000000112c30a14: 0303 94d2 | 4320 a0f2 | 0319 c0f2 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb | 8850 0054 - 0x0000000112c30a34: 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 | e363 40f9 | 6170 40b9 - 0x0000000112c30a54: 21f0 7dd3 - - 0x0000000112c30a58: ; implicit exception: dispatches to 0x0000000112c31448 - 0x0000000112c30a58: 3f00 40f9 | e203 01aa - - 0x0000000112c30a60: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30a60: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6890 42f9 | 0805 0091 | 6890 02f9 | e267 40f9 | e017 0fa9 - 0x0000000112c30a80: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } - ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@159 (line 230) - ; {optimized virtual_call} - 0x0000000112c30a80: b4f1 9a95 | e27b 40f9 - - 0x0000000112c30a88: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30a88: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28ac 42f9 | 0805 0091 | 28ac 02f9 | e203 00aa | e17b 40f9 - 0x0000000112c30aa8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@162 (line 230) - ; {optimized virtual_call} - 0x0000000112c30aa8: 7adf 9b95 - - 0x0000000112c30aac: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30aac: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2120 0791 - - 0x0000000112c30abc: ; {oop(a 'java/lang/invoke/BoundMethodHandle$Species_L'{0x000000071276b5d0})} - 0x0000000112c30abc: 00ba 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c30adc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30adc: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 28b4 42f9 | 0805 0091 | 28b4 02f9 - - 0x0000000112c30af4: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30af4: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 22ac 40b9 | 4208 0011 | 22ac 00b9 | 424c 1f12 | 5f00 0071 - 0x0000000112c30b14: c049 0054 - - 0x0000000112c30b18: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c30b18: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 2100 0691 | 8000 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c30b38: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30b38: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2894 40f9 | 0805 0091 | 2894 00f9 - - 0x0000000112c30b50: ; {metadata(method data for {method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c30b50: 0192 80d2 | 0168 a5f2 | 2100 c0f2 | 20ac 40b9 | 0008 0011 | 20ac 00b9 | 004c 1f12 | 1f00 0071 - 0x0000000112c30b70: e047 0054 - - 0x0000000112c30b74: ; {metadata(method data for {method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c30b74: 0157 94d2 | 4163 a5f2 | 2100 c0f2 | 2180 0591 - - 0x0000000112c30b84: ; {oop(a 'org/apache/comet/vector/NativeUtil$$Lambda$3997+0x000000c801e1d000'{0x000000071276b5f0})} - 0x0000000112c30b84: 00be 96d2 | c04e a2f2 | e000 c0f2 | e203 00aa | 8200 00b5 | 2900 40f9 | 2901 40b2 | 2900 00f9 - 0x0000000112c30ba4: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30ba4: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 21e0 1591 | e203 00aa | a200 00b5 | 2900 40f9 | 2901 40b2 - 0x0000000112c30bc4: 2900 00f9 | 1700 0014 - - 0x0000000112c30bcc: ; {metadata('org/apache/comet/vector/NativeUtil$$Lambda$3997+0x000000c801e1d000')} - 0x0000000112c30bcc: 0200 9ad2 | 223c a0f2 | 0219 c0f2 | 2900 40f9 | 4200 09ca | 48f4 7e92 | 0802 00b4 | e201 0837 - 0x0000000112c30bec: a901 00b4 | 3f05 00f1 | 6001 0054 | bf39 03d5 | 4200 09ca | 2900 40f9 | 4200 09ca | 48f4 7e92 - 0x0000000112c30c0c: c800 00b4 | 2900 40f9 | 2901 7fb2 | 2900 00f9 | 0200 0014 | 2200 00f9 | e273 40f9 | 4270 40b9 - 0x0000000112c30c2c: 42f0 7dd3 | e477 40f9 | 8170 40b9 | 21f0 7dd3 - - 0x0000000112c30c3c: ; implicit exception: dispatches to 0x0000000112c3148c - 0x0000000112c30c3c: 3f00 40f9 | e303 01aa - - 0x0000000112c30c44: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30c44: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 88c4 42f9 | 0805 0091 | 88c4 02f9 | e203 10a9 - - 0x0000000112c30c60: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop [256]=Oop } - ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@176 (line 230) - ; {optimized virtual_call} - 0x0000000112c30c60: fcd6 c595 | e183 40f9 - - 0x0000000112c30c68: ; implicit exception: dispatches to 0x0000000112c31490 - 0x0000000112c30c68: 3f00 40f9 | e203 01aa - - 0x0000000112c30c70: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30c70: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e0 42f9 | 0805 0091 | 68e0 02f9 | e203 00aa - - 0x0000000112c30c8c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop } - ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@179 (line 230) - ; {optimized virtual_call} - 0x0000000112c30c8c: b11f bf95 | e27b 40f9 - - 0x0000000112c30c94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30c94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68fc 42f9 | 0805 0091 | 68fc 02f9 | e287 40f9 | e303 00aa - 0x0000000112c30cb4: e17b 40f9 - - 0x0000000112c30cb8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop } - ;*invokevirtual map {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@182 (line 230) - ; {optimized virtual_call} - 0x0000000112c30cb8: 7622 bf95 | 0001 00b5 - - 0x0000000112c30cc0: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30cc0: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68a4 5839 | 0801 40b2 | 68a4 1839 | 3700 0014 - - 0x0000000112c30cdc: ; {metadata({type array long})} - 0x0000000112c30cdc: 0130 82d2 | 0100 a0f2 | 0119 c0f2 | 0308 40b9 | 0319 c0f2 | 681c 40f9 | 3f00 08eb | e104 0054 - 0x0000000112c30cfc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30cfc: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 0108 40b9 | 0119 c0f2 | 6900 1991 | 2801 40f9 | 3f00 08eb - 0x0000000112c30d1c: a100 0054 | 6824 43f9 | 0805 0091 | 6824 03f9 | 2200 0014 | 6940 1991 | 2801 40f9 | 3f00 08eb - 0x0000000112c30d3c: a100 0054 | 682c 43f9 | 0805 0091 | 682c 03f9 | 1a00 0014 | 6900 1991 | 2801 40f9 | c800 00b5 - 0x0000000112c30d5c: 2101 00f9 | e803 40b2 | 6920 1991 | 2801 00f9 | 1200 0014 | 6940 1991 | 2801 40f9 | c800 00b5 - 0x0000000112c30d7c: 2101 00f9 | e803 40b2 | 6960 1991 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c30d94: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30d94: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6818 43f9 | 0805 00d1 | 6818 03f9 | ba01 0014 | 0100 0014 - 0x0000000112c30db4: e303 00aa | e203 0032 | e04b 40f9 | 010c 40b9 | 3f84 0071 | a936 0054 | 02c4 0039 | e147 40f9 - 0x0000000112c30dd4: ; implicit exception: dispatches to 0x0000000112c314b4 - 0x0000000112c30dd4: 3f00 40f9 | e203 01aa - - 0x0000000112c30ddc: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30ddc: 04b9 91d2 | 64ca a6f2 | 2400 c0f2 | 4208 40b9 | 0219 c0f2 | 89e0 1991 | 2801 40f9 | 5f00 08eb - 0x0000000112c30dfc: a100 0054 | 8840 43f9 | 0805 0091 | 8840 03f9 | 1c00 0014 | 8920 1a91 | 2801 40f9 | 5f00 08eb - 0x0000000112c30e1c: a100 0054 | 8848 43f9 | 0805 0091 | 8848 03f9 | 1400 0014 | 89e0 1991 | 2801 40f9 | c800 00b5 - 0x0000000112c30e3c: 2201 00f9 | e803 40b2 | 8900 1a91 | 2801 00f9 | 0c00 0014 | 8920 1a91 | 2801 40f9 | c800 00b5 - 0x0000000112c30e5c: 2201 00f9 | e803 40b2 | 8940 1a91 | 2801 00f9 | 0400 0014 | 8834 43f9 | 0805 0091 | 8834 03f9 - 0x0000000112c30e7c: e27f 40f9 | 09e4 94d2 | 49cd acf2 | 8901 c0f2 - - 0x0000000112c30e8c: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [144]=Oop } - ;*invokeinterface apply {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@201 (line 232) - ; {virtual_call} - 0x0000000112c30e8c: edfd 7995 - - 0x0000000112c30e90: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30e90: 01b9 91d2 | 61ca a6f2 | 2100 c0f2 | 2850 43f9 | 0805 0091 | 2850 03f9 | e103 00aa - - 0x0000000112c30eac: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [144]=Oop } - ;*invokestatic unboxToLong {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@206 (line 232) - ; {static_call} - 0x0000000112c30eac: 056f a095 | 0200 8092 | 5f00 00eb - - 0x0000000112c30eb8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30eb8: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 08d6 80d2 | 09d8 80d2 | 0311 899a | 4168 63f8 | 2104 0091 - 0x0000000112c30ed8: 4168 23f8 | 6102 0054 | e04b 40f9 - - 0x0000000112c30ee4: ; {oop(a 'java/lang/Class'{0x0000000702f28118} = 'scala/None$')} - 0x0000000112c30ee4: 0223 90d2 | 425e a0f2 | e200 c0f2 | 4270 40b9 | 42f0 7dd3 | e303 0032 | 010c 40b9 | 3f88 0071 - 0x0000000112c30f04: a92d 0054 | 03c8 0039 | e003 02aa | fd7b 54a9 | ff43 0591 - - 0x0000000112c30f18: ; {poll_return} - 0x0000000112c30f18: 88a3 41f9 | ff63 28eb | 882d 0054 | c003 5fd6 - - 0x0000000112c30f28: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30f28: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 4368 43f9 | 6304 0091 | 4368 03f9 | e08b 00f9 | e567 40f9 - 0x0000000112c30f48: e05b 40f9 | e13f 40f9 | e203 01aa - - 0x0000000112c30f54: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c30f54: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 - - 0x0000000112c30f60: ; {metadata('org/apache/comet/vector/NativeUtil')} - 0x0000000112c30f60: 08f2 9ad2 | 483c a0f2 | 0819 c0f2 | 69e0 1b91 | 2801 00f9 | 6880 43f9 | 0805 0091 | 6880 03f9 - 0x0000000112c30f80: e203 00aa | e303 05aa - - 0x0000000112c30f88: ; ImmutableOopMap {[144]=Oop } - ;*invokevirtual importVector {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@242 (line 239) - ; {optimized virtual_call} - 0x0000000112c30f88: ee29 6894 | e303 0032 | e54b 40f9 | a40c 40b9 | 9f8c 0071 | 092a 0054 | a3cc 0039 - - 0x0000000112c30fa4: ; {metadata('scala/Some')} - 0x0000000112c30fa4: 0300 90d2 | 2320 a0f2 | 0319 c0f2 | e08f 00f9 | 8087 40f9 | 0b40 0091 | 888f 40f9 | 7f01 08eb - 0x0000000112c30fc4: 8829 0054 | 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | bf3a 03d5 - - 0x0000000112c30fe0: ; {metadata('org/apache/spark/sql/vectorized/ColumnarBatch')} - 0x0000000112c30fe0: 03f6 82d2 | e342 a0f2 | 0319 c0f2 | e097 00f9 | 8087 40f9 | 0b60 0091 | 888f 40f9 | 7f01 08eb - 0x0000000112c31000: e827 0054 | 8b87 00f9 | ea03 40b2 | 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f01 00f9 - 0x0000000112c31020: bf3a 03d5 - - 0x0000000112c31024: ; {oop(a 'java/lang/Class'{0x00000007039c31e8} = 'scala/reflect/ClassTag$')} - 0x0000000112c31024: 023d 86d2 | 8273 a0f2 | e200 c0f2 | 4170 40b9 | 21f0 7dd3 - - 0x0000000112c31038: ; implicit exception: dispatches to 0x0000000112c31504 - 0x0000000112c31038: 3f00 40f9 | e203 01aa - - 0x0000000112c31040: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31040: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6890 43f9 | 0805 0091 | 6890 03f9 - - 0x0000000112c31058: ; {oop(a 'java/lang/Class'{0x0000000712525738} = 'org/apache/spark/sql/vectorized/ColumnVector')} - 0x0000000112c31058: 02e7 8ad2 | 424a a2f2 | e200 c0f2 | e093 00f9 - - 0x0000000112c31068: ; ImmutableOopMap {[280]=Oop [144]=Oop [296]=Oop [288]=Oop } - ;*invokevirtual apply {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@269 (line 240) - ; {optimized virtual_call} - 0x0000000112c31068: 8aa7 b395 | e18f 40f9 - - 0x0000000112c31070: ; implicit exception: dispatches to 0x0000000112c31508 - 0x0000000112c31070: 3f00 40f9 | e203 01aa - - 0x0000000112c31078: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31078: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 4208 40b9 | 0219 c0f2 | 69a0 1d91 | 2801 40f9 | 5f00 08eb - 0x0000000112c31098: a100 0054 | 68b8 43f9 | 0805 0091 | 68b8 03f9 | 1c00 0014 | 69e0 1d91 | 2801 40f9 | 5f00 08eb - 0x0000000112c310b8: a100 0054 | 68c0 43f9 | 0805 0091 | 68c0 03f9 | 1400 0014 | 69a0 1d91 | 2801 40f9 | c800 00b5 - 0x0000000112c310d8: 2201 00f9 | e803 40b2 | 69c0 1d91 | 2801 00f9 | 0c00 0014 | 69e0 1d91 | 2801 40f9 | c800 00b5 - 0x0000000112c310f8: 2201 00f9 | e803 40b2 | 6900 1e91 | 2801 00f9 | 0400 0014 | 68ac 43f9 | 0805 0091 | 68ac 03f9 - 0x0000000112c31118: e203 00aa | 09f4 95d2 | e921 a0f2 | 0919 c0f2 - - 0x0000000112c31128: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } - ;*invokeinterface toArray {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@272 (line 240) - ; {virtual_call} - 0x0000000112c31128: c6ba b995 | 0001 00b5 - - 0x0000000112c31130: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31130: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 4824 5e39 | 0801 40b2 | 4824 1e39 | 3700 0014 - - 0x0000000112c3114c: ; {metadata('org/apache/spark/sql/vectorized/ColumnVector'[])} - 0x0000000112c3114c: 01f7 9ad2 | a144 a0f2 | 0119 c0f2 | 0208 40b9 | 0219 c0f2 | 4820 40f9 | 3f00 08eb | e104 0054 - 0x0000000112c3116c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3116c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 0108 40b9 | 0119 c0f2 | 4980 1e91 | 2801 40f9 | 3f00 08eb - 0x0000000112c3118c: a100 0054 | 48d4 43f9 | 0805 0091 | 48d4 03f9 | 2200 0014 | 49c0 1e91 | 2801 40f9 | 3f00 08eb - 0x0000000112c311ac: a100 0054 | 48dc 43f9 | 0805 0091 | 48dc 03f9 | 1a00 0014 | 4980 1e91 | 2801 40f9 | c800 00b5 - 0x0000000112c311cc: 2101 00f9 | e803 40b2 | 49a0 1e91 | 2801 00f9 | 1200 0014 | 49c0 1e91 | 2801 40f9 | c800 00b5 - 0x0000000112c311ec: 2101 00f9 | e803 40b2 | 49e0 1e91 | 2801 00f9 | 0a00 0014 | 0900 0014 - - 0x0000000112c31204: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31204: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 48c8 43f9 | 0805 00d1 | 48c8 03f9 | bc00 0014 | 0100 0014 - 0x0000000112c31224: e203 00aa | e093 40f9 - - 0x0000000112c3122c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3122c: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 68e4 43f9 | 0805 0091 | 68e4 03f9 | e08b 40f9 | 007c 40d3 - 0x0000000112c3124c: e303 00aa | e193 40f9 - - 0x0000000112c31254: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@283 (line 240) - ; {optimized virtual_call} - 0x0000000112c31254: 1b73 dc97 | e097 40f9 - - 0x0000000112c3125c: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c3125c: 02b9 91d2 | 62ca a6f2 | 2200 c0f2 | 48ec 43f9 | 0805 0091 | 48ec 03f9 | e207 52a9 - - 0x0000000112c31278: ; ImmutableOopMap {[144]=Oop [296]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@286 (line 240) - ; {optimized virtual_call} - 0x0000000112c31278: 16b7 9a95 | e003 0032 | e54b 40f9 | a30c 40b9 | 7f90 0071 | 4914 0054 | a0d0 0039 | e097 40f9 - 0x0000000112c31298: fd7b 54a9 | ff43 0591 - - 0x0000000112c312a0: ; {poll_return} - 0x0000000112c312a0: 88a3 41f9 | ff63 28eb | 2814 0054 | c003 5fd6 | e04f 00f9 | e54b 40f9 - - 0x0000000112c312b8: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c312b8: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 6030 41f9 | 0004 0091 | 6030 01f9 | e24f 40f9 - - 0x0000000112c312d4: ; ImmutableOopMap {[144]=Oop c_rarg2=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) - ; {runtime_call load_mirror_patching Runtime1 stub} - 0x0000000112c312d4: 4b40 7195 | 68ec 4439 | 1f11 0071 - - 0x0000000112c312e0: ; implicit exception: dispatches to 0x0000000112c31538 - 0x0000000112c312e0: e112 0054 | 8087 40f9 | 0bc0 0091 | 888f 40f9 | 7f01 08eb | 4812 0054 | 8b87 00f9 | ea03 40b2 - 0x0000000112c31300: 0a00 00f9 | ea03 032a | 0a7c 0129 | 0a40 0091 | 5f7d 00a9 | 5f7d 01a9 | bf3a 03d5 | e103 00aa - 0x0000000112c31320: ; {metadata(method data for {method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31320: 03b9 91d2 | 63ca a6f2 | 2300 c0f2 | 683c 41f9 | 0805 0091 | 683c 01f9 | e103 00aa | e09b 00f9 - 0x0000000112c31340: ; ImmutableOopMap {[144]=Oop [304]=Oop } - ;*invokespecial {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@60 (line 227) - ; {optimized virtual_call} - 0x0000000112c31340: f06e 6e95 | e003 0032 | e14b 40f9 - - 0x0000000112c3134c: ; implicit exception: dispatches to 0x0000000112c31544 - 0x0000000112c3134c: 220c 40b9 | 5f78 0071 | a90f 0054 | 20b8 0039 | e09b 40f9 | 8500 0014 - - 0x0000000112c31364: ; {metadata({method} {0x0000000135fed7d0} 'getNextBatch' '(ILscala/Function2;)Lscala/Option;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31364: 08fa 9ad2 | c8bf a6f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c3137c: ; ImmutableOopMap {c_rarg1=Oop c_rarg3=Oop [136]=Oop } - ;*synchronization entry - ; - org.apache.comet.vector.NativeUtil::getNextBatch@-1 - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c3137c: 6142 7195 | affa ff17 - - 0x0000000112c31384: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop c_rarg0=Oop } - ;*invokevirtual _1 {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@19 (line 227) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31384: df2e 7195 | e803 00aa - - 0x0000000112c3138c: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [152]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@22 (line 227) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c3138c: dd37 7195 | e803 00aa - - 0x0000000112c31394: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@32 (line 227) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c31394: db37 7195 - - 0x0000000112c31398: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [160]=Oop c_rarg1=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@37 (line 227) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c31398: 1a31 7195 | b9fb ff17 | e803 00aa - - 0x0000000112c313a4: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [168]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@77 (line 227) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c313a4: d737 7195 | e803 00aa - - 0x0000000112c313ac: ; ImmutableOopMap {[120]=Oop [136]=Oop [144]=Oop [176]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@87 (line 227) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c313ac: d537 7195 - - 0x0000000112c313b0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@97 (line 227) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c313b0: d42e 7195 | e813 40b2 | e903 00aa - - 0x0000000112c313bc: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c313bc: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c313c8: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@97 (line 227) - 0x0000000112c313c8: c003 3fd6 - - 0x0000000112c313cc: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop [144]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@98 (line 229) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c313cc: 0d31 7195 | a7fc ff17 - - 0x0000000112c313d4: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop c_rarg5=Oop [144]=Oop c_rarg0=Oop c_rarg3=Oop c_rarg1=Oop } - ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@110 (line 229) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c313d4: cb2e 7195 - - 0x0000000112c313d8: ; {metadata({method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c313d8: 08ed 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c313f0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop c_rarg0=Oop } - ;*synchronization entry - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 - ; - org.apache.comet.vector.NativeUtil::getNextBatch@116 (line 229) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c313f0: 4442 7195 | d4fc ff17 - - 0x0000000112c313f8: ; {metadata({method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c313f8: 084d 80d2 | 0868 a5f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c31410: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop } - ;*synchronization entry - ; - java.lang.invoke.LambdaForm$MH/0x000000c801000400::invoke@-1 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@4 - ; - org.apache.comet.vector.NativeUtil::getNextBatch@116 (line 229) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c31410: 3c42 7195 | e3fc ff17 - - 0x0000000112c31418: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop c_rarg0=Oop c_rarg2=Oop c_rarg3=Oop c_rarg4=Oop c_rarg1=Oop } - ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@127 (line 229) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31418: ba2e 7195 - - 0x0000000112c3141c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [184]=Oop [192]=Oop [200]=Oop [224]=Oop [216]=Oop [232]=Oop c_rarg0=Oop c_rarg3=Oop } - ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@130 (line 229) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c3141c: b92e 7195 | e803 00aa - - 0x0000000112c31424: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [144]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@136 (line 229) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c31424: b737 7195 | e803 7bb2 | e903 00aa - - 0x0000000112c31430: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c31430: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c3143c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop c_rarg0=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@146 (line 229) - 0x0000000112c3143c: c003 3fd6 - - 0x0000000112c31440: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [192]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop [144]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@147 (line 230) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c31440: f030 7195 | 82fd ff17 - - 0x0000000112c31448: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop c_rarg5=Oop [144]=Oop c_rarg0=Oop c_rarg1=Oop } - ;*invokevirtual refArrayOps {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@159 (line 230) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31448: ae2e 7195 - - 0x0000000112c3144c: ; {metadata({method} {0x000000c8000d7f68} 'linkToTargetMethod' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/Invokers$Holder')} - 0x0000000112c3144c: 08ed 8fd2 | a801 a0f2 | 0819 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c31464: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop c_rarg0=Oop } - ;*synchronization entry - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@-1 - ; - org.apache.comet.vector.NativeUtil::getNextBatch@165 (line 230) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c31464: 2742 7195 | acfd ff17 - - 0x0000000112c3146c: ; {metadata({method} {0x000000012b400268} 'invoke' '(Ljava/lang/Object;)Ljava/lang/Object;' in 'java/lang/invoke/LambdaForm$MH+0x000000c801000400')} - 0x0000000112c3146c: 084d 80d2 | 0868 a5f2 | 2800 c0f2 | e807 00f9 | 0800 8092 | e803 00f9 - - 0x0000000112c31484: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [224]=Oop [232]=Oop [248]=Oop [144]=Oop [240]=Oop } - ;*synchronization entry - ; - java.lang.invoke.LambdaForm$MH/0x000000c801000400::invoke@-1 - ; - java.lang.invoke.Invokers$Holder::linkToTargetMethod@4 - ; - org.apache.comet.vector.NativeUtil::getNextBatch@165 (line 230) - ; {runtime_call counter_overflow Runtime1 stub} - 0x0000000112c31484: 1f42 7195 | bbfd ff17 - - 0x0000000112c3148c: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop c_rarg0=Oop c_rarg2=Oop c_rarg1=Oop } - ;*invokevirtual Long {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@176 (line 230) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c3148c: 9d2e 7195 - - 0x0000000112c31490: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop [240]=Oop [264]=Oop c_rarg0=Oop c_rarg1=Oop } - ;*invokevirtual canBuildFrom {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@179 (line 230) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31490: 9c2e 7195 | e803 00aa - - 0x0000000112c31498: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop [144]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@185 (line 230) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c31498: 9a37 7195 | 2804 80d2 | e903 00aa - - 0x0000000112c314a4: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c314a4: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c314b0: ; ImmutableOopMap {[120]=Oop [136]=Oop [176]=Oop [200]=Oop [248]=Oop c_rarg3=Oop c_rarg0=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@195 (line 230) - 0x0000000112c314b0: c003 3fd6 - - 0x0000000112c314b4: ; ImmutableOopMap {[120]=Oop [176]=Oop [200]=Oop [248]=Oop c_rarg3=Oop c_rarg0=Oop [144]=Oop c_rarg1=Oop [136]=Oop } - ;*invokeinterface apply {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@201 (line 232) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c314b4: 932e 7195 | 4804 80d2 | e903 00aa - - 0x0000000112c314c0: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c314c0: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c314cc: ; ImmutableOopMap {c_rarg0=Oop [144]=Oop c_rarg2=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@232 (line 237) - 0x0000000112c314cc: c003 3fd6 - - 0x0000000112c314d0: ; {internal_word} - 0x0000000112c314d0: 48d2 ff10 | 88af 01f9 - - 0x0000000112c314d8: ; {runtime_call SafepointBlob} - 0x0000000112c314d8: 4a87 6e15 | 6804 80d2 | e903 05aa - - 0x0000000112c314e4: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c314e4: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c314f0: ; ImmutableOopMap {c_rarg0=Oop c_rarg5=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@252 (line 239) - 0x0000000112c314f0: c003 3fd6 - - 0x0000000112c314f4: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@253 (line 240) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c314f4: c330 7195 | bafe ff17 - - 0x0000000112c314fc: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop [296]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@257 (line 240) - ; {runtime_call fast_new_instance Runtime1 stub} - 0x0000000112c314fc: c130 7195 | c9fe ff17 - - 0x0000000112c31504: ; ImmutableOopMap {c_rarg5=Oop [144]=Oop [280]=Oop [296]=Oop c_rarg0=Oop c_rarg1=Oop } - ;*invokevirtual apply {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@269 (line 240) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31504: 7f2e 7195 - - 0x0000000112c31508: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop c_rarg0=Oop c_rarg1=Oop } - ;*invokeinterface toArray {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@272 (line 240) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31508: 7e2e 7195 | e803 00aa - - 0x0000000112c31510: ; ImmutableOopMap {[144]=Oop [296]=Oop [288]=Oop } - ;*checkcast {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@277 (line 240) - ; {runtime_call throw_class_cast_exception Runtime1 stub} - 0x0000000112c31510: 7c37 7195 | 8804 80d2 | e903 05aa - - 0x0000000112c3151c: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c3151c: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c31528: ; ImmutableOopMap {[296]=Oop c_rarg5=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@294 (line 240) - 0x0000000112c31528: c003 3fd6 - - 0x0000000112c3152c: ; {internal_word} - 0x0000000112c3152c: a8eb ff10 | 88af 01f9 - - 0x0000000112c31534: ; {runtime_call SafepointBlob} - 0x0000000112c31534: 3387 6e15 - - 0x0000000112c31538: ; ImmutableOopMap {c_rarg2=Oop [144]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31538: 722e 7195 - - 0x0000000112c3153c: ; ImmutableOopMap {c_rarg2=Oop [144]=Oop } - ;*new {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@54 (line 227) - ; {runtime_call fast_new_instance_init_check Runtime1 stub} - 0x0000000112c3153c: 7131 7195 | 77ff ff17 - - 0x0000000112c31544: ; ImmutableOopMap {[304]=Oop c_rarg1=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@68 (line 227) - ; {runtime_call throw_null_pointer_exception Runtime1 stub} - 0x0000000112c31544: 6f2e 7195 | e80f 7fb2 | e903 01aa - - 0x0000000112c31550: ; {runtime_call throw_range_check_failed Runtime1 stub} - 0x0000000112c31550: 1e60 98d2 | fe10 a3f2 | 3e00 c0f2 - - 0x0000000112c3155c: ; ImmutableOopMap {[304]=Oop c_rarg1=Oop [144]=Oop } - ;*bastore {reexecute=0 rethrow=0 return_oop=0} - ; - org.apache.comet.vector.NativeUtil::getNextBatch@68 (line 227) - 0x0000000112c3155c: c003 3fd6 | 1f20 03d5 | 1f20 03d5 | 80eb 41f9 | 9feb 01f9 | 9fef 01f9 | fd7b 54a9 | ff43 0591 - 0x0000000112c3157c: ; {runtime_call unwind_exception Runtime1 stub} - 0x0000000112c3157c: 61f2 6e15 -[Stub Code] - 0x0000000112c31580: ; {no_reloc} - 0x0000000112c31580: df3f 03d5 - - 0x0000000112c31584: ; {metadata(NULL)} - 0x0000000112c31584: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c315a0: ; {trampoline_stub} - 0x0000000112c315a0: 4800 0058 | 0001 1fd6 | 4068 9719 | 0100 0000 - - 0x0000000112c315b0: ; {static_stub} - 0x0000000112c315b0: df3f 03d5 - - 0x0000000112c315b4: ; {metadata(NULL)} - 0x0000000112c315b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c315d0: ; {trampoline_stub} - 0x0000000112c315d0: 4800 0058 | 0001 1fd6 | c0cb c212 | 0100 0000 - - 0x0000000112c315e0: ; {static_stub} - 0x0000000112c315e0: df3f 03d5 - - 0x0000000112c315e4: ; {metadata(NULL)} - 0x0000000112c315e4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31600: ; {trampoline_stub} - 0x0000000112c31600: 4800 0058 | 0001 1fd6 | c08e f818 | 0100 0000 - - 0x0000000112c31610: ; {static_stub} - 0x0000000112c31610: df3f 03d5 - - 0x0000000112c31614: ; {metadata(NULL)} - 0x0000000112c31614: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31630: ; {trampoline_stub} - 0x0000000112c31630: 4800 0058 | 0001 1fd6 | c0b4 2719 | 0100 0000 - - 0x0000000112c31640: ; {static_stub} - 0x0000000112c31640: df3f 03d5 - - 0x0000000112c31644: ; {metadata(NULL)} - 0x0000000112c31644: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31660: ; {trampoline_stub} - 0x0000000112c31660: 4800 0058 | 0001 1fd6 | 5062 2719 | 0100 0000 - - 0x0000000112c31670: ; {static_stub} - 0x0000000112c31670: df3f 03d5 - - 0x0000000112c31674: ; {metadata(NULL)} - 0x0000000112c31674: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31690: ; {trampoline_stub} - 0x0000000112c31690: 4800 0058 | 0001 1fd6 | d08e f818 | 0100 0000 - - 0x0000000112c316a0: ; {static_stub} - 0x0000000112c316a0: df3f 03d5 - - 0x0000000112c316a4: ; {metadata(NULL)} - 0x0000000112c316a4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c316c0: ; {trampoline_stub} - 0x0000000112c316c0: 4800 0058 | 0001 1fd6 | d0b4 2719 | 0100 0000 - - 0x0000000112c316d0: ; {static_stub} - 0x0000000112c316d0: df3f 03d5 - - 0x0000000112c316d4: ; {metadata(NULL)} - 0x0000000112c316d4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c316f0: ; {trampoline_stub} - 0x0000000112c316f0: 4800 0058 | 0001 1fd6 | 50d1 2e19 | 0100 0000 - - 0x0000000112c31700: ; {static_stub} - 0x0000000112c31700: df3f 03d5 - - 0x0000000112c31704: ; {metadata(NULL)} - 0x0000000112c31704: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31720: ; {trampoline_stub} - 0x0000000112c31720: 4800 0058 | 0001 1fd6 | 9088 3219 | 0100 0000 - - 0x0000000112c31730: ; {static_stub} - 0x0000000112c31730: df3f 03d5 - - 0x0000000112c31734: ; {metadata(NULL)} - 0x0000000112c31734: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31750: ; {trampoline_stub} - 0x0000000112c31750: 4800 0058 | 0001 1fd6 | 5068 da19 | 0100 0000 - - 0x0000000112c31760: ; {static_stub} - 0x0000000112c31760: df3f 03d5 - - 0x0000000112c31764: ; {metadata(NULL)} - 0x0000000112c31764: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31780: ; {trampoline_stub} - 0x0000000112c31780: 4800 0058 | 0001 1fd6 | 508b bf19 | 0100 0000 - - 0x0000000112c31790: ; {static_stub} - 0x0000000112c31790: df3f 03d5 - - 0x0000000112c31794: ; {metadata(NULL)} - 0x0000000112c31794: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c317b0: ; {trampoline_stub} - 0x0000000112c317b0: 4800 0058 | 0001 1fd6 | 9096 bf19 | 0100 0000 - - 0x0000000112c317c0: ; {static_stub} - 0x0000000112c317c0: df3f 03d5 - - 0x0000000112c317c4: ; {metadata(NULL)} - 0x0000000112c317c4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c317e0: ; {trampoline_stub} - 0x0000000112c317e0: 4800 0058 | 0001 1fd6 | 50d1 2e19 | 0100 0000 - - 0x0000000112c317f0: ; {static_stub} - 0x0000000112c317f0: df3f 03d5 - - 0x0000000112c317f4: ; {metadata(NULL)} - 0x0000000112c317f4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31810: ; {trampoline_stub} - 0x0000000112c31810: 4800 0058 | 0001 1fd6 | 9088 3219 | 0100 0000 - - 0x0000000112c31820: ; {static_stub} - 0x0000000112c31820: df3f 03d5 - - 0x0000000112c31824: ; {metadata(NULL)} - 0x0000000112c31824: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31840: ; {trampoline_stub} - 0x0000000112c31840: 4800 0058 | 0001 1fd6 | 5068 da19 | 0100 0000 - - 0x0000000112c31850: ; {static_stub} - 0x0000000112c31850: df3f 03d5 - - 0x0000000112c31854: ; {metadata(NULL)} - 0x0000000112c31854: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31870: ; {trampoline_stub} - 0x0000000112c31870: 4800 0058 | 0001 1fd6 | 508b bf19 | 0100 0000 - - 0x0000000112c31880: ; {static_stub} - 0x0000000112c31880: df3f 03d5 - - 0x0000000112c31884: ; {metadata(NULL)} - 0x0000000112c31884: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c318a0: ; {trampoline_stub} - 0x0000000112c318a0: 4800 0058 | 0001 1fd6 | 9096 bf19 | 0100 0000 - - 0x0000000112c318b0: ; {static_stub} - 0x0000000112c318b0: df3f 03d5 - - 0x0000000112c318b4: ; {metadata(NULL)} - 0x0000000112c318b4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c318d0: ; {trampoline_stub} - 0x0000000112c318d0: 4800 0058 | 0001 1fd6 | 4006 ab18 | 0100 0000 - - 0x0000000112c318e0: ; {static_stub} - 0x0000000112c318e0: df3f 03d5 - - 0x0000000112c318e4: ; {metadata(NULL)} - 0x0000000112c318e4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31900: ; {trampoline_stub} - 0x0000000112c31900: 4800 0058 | 0001 1fd6 | c0ca 4419 | 0100 0000 - - 0x0000000112c31910: ; {static_stub} - 0x0000000112c31910: df3f 03d5 - - 0x0000000112c31914: ; {metadata({method} {0x0000000135fed960} 'importVector' '([Lorg/apache/arrow/c/ArrowArray;[Lorg/apache/arrow/c/ArrowSchema;)Lscala/collection/Seq;' in 'org/apache/comet/vector/NativeUtil')} - 0x0000000112c31914: 0c2c 9bd2 | ccbf a6f2 | 2c00 c0f2 | 88e7 92d2 | 880f a3f2 | 2800 c0f2 | 0001 1fd6 - - 0x0000000112c31930: ; {trampoline_stub} - 0x0000000112c31930: 4800 0058 | 0001 1fd6 | 40b7 6314 | 0100 0000 - - 0x0000000112c31940: ; {static_stub} - 0x0000000112c31940: df3f 03d5 - - 0x0000000112c31944: ; {metadata(NULL)} - 0x0000000112c31944: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31960: ; {trampoline_stub} - 0x0000000112c31960: 4800 0058 | 0001 1fd6 | 90ae 9119 | 0100 0000 - - 0x0000000112c31970: ; {static_stub} - 0x0000000112c31970: df3f 03d5 - - 0x0000000112c31974: ; {metadata(NULL)} - 0x0000000112c31974: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31990: ; {trampoline_stub} - 0x0000000112c31990: 4800 0058 | 0001 1fd6 | 40fc a919 | 0100 0000 - - 0x0000000112c319a0: ; {static_stub} - 0x0000000112c319a0: df3f 03d5 - - 0x0000000112c319a4: ; {metadata({method} {0x0000000135f02840} '' '([Lorg/apache/spark/sql/vectorized/ColumnVector;I)V' in 'org/apache/spark/sql/vectorized/ColumnarBatch')} - 0x0000000112c319a4: 0c08 85d2 | 0cbe a6f2 | 2c00 c0f2 | 8887 8fd2 | 880f a3f2 | 2800 c0f2 | 0001 1fd6 - - 0x0000000112c319c0: ; {trampoline_stub} - 0x0000000112c319c0: 4800 0058 | 0001 1fd6 | c0de 3412 | 0100 0000 - - 0x0000000112c319d0: ; {static_stub} - 0x0000000112c319d0: df3f 03d5 - - 0x0000000112c319d4: ; {metadata(NULL)} - 0x0000000112c319d4: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c319f0: ; {trampoline_stub} - 0x0000000112c319f0: 4800 0058 | 0001 1fd6 | d0ee 2d19 | 0100 0000 - - 0x0000000112c31a00: ; {static_stub} - 0x0000000112c31a00: df3f 03d5 - - 0x0000000112c31a04: ; {metadata(NULL)} - 0x0000000112c31a04: 0c00 80d2 | 0c00 a0f2 | 0c00 c0f2 | 0800 80d2 | 0800 a0f2 | 0800 c0f2 | 0001 1fd6 - - 0x0000000112c31a20: ; {trampoline_stub} - 0x0000000112c31a20: 4800 0058 | 0001 1fd6 | 00cf 7c18 | 0100 0000 -[Exception Handler] - 0x0000000112c31a30: ; {runtime_call handle_exception_from_callee Runtime1 stub} - 0x0000000112c31a30: b434 7195 | c1d5 bbd4 | 3127 2a09 | 0100 0000 -[Deopt Handler Code] - 0x0000000112c31a40: 1e00 0010 - - 0x0000000112c31a44: ; {runtime_call DeoptimizationBlob} - 0x0000000112c31a44: bf86 6e15 -[/MachCode] - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x0000000c638ca180, length=286, elements={ -0x00000001033f4320, 0x00000001034028a0, 0x0000000c688a4000, 0x0000000c688a4c00, -0x0000000c688a5200, 0x0000000c688a5800, 0x0000000c688a5e00, 0x0000000c688a6400, -0x0000000c688a6a00, 0x0000000c688a7000, 0x0000000c6890a400, 0x0000000c6862c600, -0x0000000c66c26a00, 0x0000000c687c5800, 0x0000000c687c5200, 0x0000000c687c4c00, -0x0000000c687c4600, 0x0000000c687c5e00, 0x0000000c66cbc000, 0x0000000c66cbc600, -0x0000000c66cbcc00, 0x0000000c66cbd200, 0x0000000c66cbd800, 0x0000000c66cbde00, -0x0000000c66cbe400, 0x0000000c66cbea00, 0x0000000c66ccd200, 0x0000000c66ccf000, -0x0000000c66ccf600, 0x0000000c66d1c600, 0x0000000c66d1cc00, 0x0000000c66d1d800, -0x0000000c66d1de00, 0x0000000c66d1e400, 0x0000000c66d1f000, 0x0000000c66d1f600, -0x0000000c66d68c00, 0x0000000c66d69200, 0x0000000c66d69e00, 0x0000000c66d6aa00, -0x0000000c66d6b000, 0x0000000c66d6b600, 0x0000000c66dd8000, 0x0000000c66de4600, -0x0000000c66de5e00, 0x0000000c66ec1800, 0x0000000c66530c00, 0x0000000c67a99800, -0x0000000c67a99e00, 0x0000000c67a9a400, 0x0000000c67a9aa00, 0x0000000c67a9b000, -0x0000000c676b0600, 0x0000000c676b2400, 0x0000000c67aab600, 0x0000000c67aaaa00, -0x0000000c67130600, 0x0000000c67130c00, 0x0000000c67131200, 0x0000000c67131800, -0x0000000c65d64600, 0x0000000c65d64c00, 0x0000000c65d65200, 0x0000000c65d65800, -0x0000000c65d65e00, 0x0000000c65e5a400, 0x0000000c65e59e00, 0x0000000c65e5aa00, -0x0000000c65e5b000, 0x0000000c65e5b600, 0x0000000c64688000, 0x0000000c6862f000, -0x0000000c677e6a00, 0x0000000c677e4000, 0x0000000c677e4c00, 0x0000000c67132a00, -0x0000000c654ad200, 0x0000000c654ad800, 0x0000000c654ade00, 0x0000000c654aea00, -0x0000000c654af000, 0x0000000c654af600, 0x0000000c688a7600, 0x0000000c677e5800, -0x0000000c677e5e00, 0x0000000c677e7000, 0x0000000c677e7600, 0x0000000c67aaa400, -0x0000000c6890b600, 0x0000000c68908000, 0x0000000c65e59200, 0x0000000c646f8c00, -0x0000000c646f8600, 0x0000000c640f0000, 0x0000000c640f0600, 0x0000000c640f0c00, -0x0000000c640f1200, 0x0000000c640f1800, 0x0000000c640f1e00, 0x0000000c640f2400, -0x0000000c640f2a00, 0x0000000c640f3000, 0x0000000c640f3600, 0x0000000c667b4000, -0x0000000c667b4600, 0x0000000c667b4c00, 0x0000000c667b5200, 0x0000000c667b5800, -0x0000000c667b5e00, 0x0000000c667b6400, 0x0000000c667b6a00, 0x0000000c667b7000, -0x0000000c667b7600, 0x0000000c6557c000, 0x0000000c6557c600, 0x0000000c6557cc00, -0x0000000c6557d200, 0x0000000c6557d800, 0x0000000c6557de00, 0x0000000c6557e400, -0x0000000c6557ea00, 0x0000000c6557f000, 0x0000000c6557f600, 0x0000000c640fc000, -0x0000000c640fc600, 0x0000000c640fcc00, 0x0000000c640fd200, 0x0000000c640fd800, -0x0000000c640ff000, 0x0000000c655bd200, 0x0000000c655be400, 0x0000000c655bea00, -0x0000000c655bf000, 0x0000000c655bf600, 0x0000000c66de6a00, 0x0000000c655e4000, -0x0000000c655e4600, 0x0000000c655e4c00, 0x0000000c655e5200, 0x0000000c655e5800, -0x0000000c655e5e00, 0x0000000c655e6400, 0x0000000c655e6a00, 0x0000000c655e7000, -0x0000000c655e7600, 0x0000000c64724000, 0x0000000c64724600, 0x0000000c64724c00, -0x0000000c64725200, 0x0000000c64725800, 0x0000000c64725e00, 0x0000000c64726400, -0x0000000c64726a00, 0x0000000c64727000, 0x0000000c64727600, 0x0000000c6412c000, -0x0000000c6412c600, 0x0000000c6412cc00, 0x0000000c6412d200, 0x0000000c6412d800, -0x0000000c6412e400, 0x0000000c6412de00, 0x0000000c6412ea00, 0x0000000c6412f000, -0x0000000c64134600, 0x0000000c64134000, 0x0000000c64134c00, 0x0000000c64135200, -0x0000000c64135800, 0x0000000c64135e00, 0x0000000c64136400, 0x0000000c64136a00, -0x0000000c64137000, 0x0000000c64137600, 0x0000000c655ec600, 0x0000000c655ec000, -0x0000000c655ecc00, 0x0000000c655ed200, 0x0000000c655ed800, 0x0000000c655ede00, -0x0000000c655ee400, 0x0000000c655eea00, 0x0000000c655ef000, 0x0000000c655ef600, -0x0000000c655f0600, 0x0000000c655f0000, 0x0000000c655f1e00, 0x0000000c655f3600, -0x0000000c64738600, 0x0000000c6473b600, 0x0000000c6473b000, 0x0000000c655f2400, -0x0000000c6412f600, 0x0000000c64754000, 0x0000000c64754600, 0x0000000c64754c00, -0x0000000c64755200, 0x0000000c64755800, 0x0000000c64755e00, 0x0000000c64756400, -0x0000000c64756a00, 0x0000000c64757000, 0x0000000c64757600, 0x0000000c64758000, -0x0000000c64758600, 0x0000000c64758c00, 0x0000000c64759200, 0x0000000c64759800, -0x0000000c64759e00, 0x0000000c6475a400, 0x0000000c6475aa00, 0x0000000c6475b000, -0x0000000c6475b600, 0x0000000c65600000, 0x0000000c65600600, 0x0000000c65601200, -0x0000000c65601800, 0x0000000c65601e00, 0x0000000c65602400, 0x0000000c65602a00, -0x0000000c65603000, 0x0000000c65603600, 0x0000000c65608000, 0x0000000c65608600, -0x0000000c65608c00, 0x0000000c65609200, 0x0000000c65609800, 0x0000000c65609e00, -0x0000000c6560a400, 0x0000000c6560aa00, 0x0000000c6560b000, 0x0000000c6560b600, -0x0000000c64170000, 0x0000000c64170600, 0x0000000c64170c00, 0x0000000c64171200, -0x0000000c64171800, 0x0000000c64171e00, 0x0000000c64172400, 0x0000000c64172a00, -0x0000000c64173000, 0x0000000c64173600, 0x0000000c64768000, 0x0000000c64768600, -0x0000000c64769200, 0x0000000c64769800, 0x0000000c641a0600, 0x0000000c641a1800, -0x0000000c641a1e00, 0x0000000c641a2400, 0x0000000c641a2a00, 0x0000000c641a3000, -0x0000000c641a3600, 0x0000000c646fb600, 0x0000000c65dbde00, 0x0000000c646e1200, -0x0000000c66531800, 0x0000000c66531200, 0x0000000c641a8000, 0x0000000c641a8600, -0x0000000c641a8c00, 0x0000000c641a9200, 0x0000000c641a9800, 0x0000000c641a9e00, -0x0000000c641aa400, 0x0000000c641aaa00, 0x0000000c641ab000, 0x0000000c641ab600, -0x0000000c6477c600, 0x0000000c6477d200, 0x0000000c6477d800, 0x0000000c6477e400, -0x0000000c6477ea00, 0x0000000c6477f600, 0x0000000c64200000, 0x0000000c64200600, -0x0000000c64200c00, 0x0000000c64201200, 0x0000000c64201800, 0x0000000c64201e00, -0x0000000c64202400, 0x0000000c64202a00, 0x0000000c64203000, 0x0000000c64203600, -0x0000000c64204000, 0x0000000c64289e00 -} - -Java Threads: ( => current thread ) - 0x00000001033f4320 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=5891, stack(0x000000016d34c000,0x000000016d74f000)] - 0x00000001034028a0 JavaThread "Reference Handler" daemon [_thread_blocked, id=30723, stack(0x000000016e3a0000,0x000000016e7a3000)] - 0x0000000c688a4000 JavaThread "Finalizer" daemon [_thread_blocked, id=22531, stack(0x000000016e7ac000,0x000000016ebaf000)] - 0x0000000c688a4c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=28675, stack(0x000000016ecd0000,0x000000016f0d3000)] - 0x0000000c688a5200 JavaThread "Service Thread" daemon [_thread_blocked, id=24579, stack(0x000000016f0dc000,0x000000016f4df000)] - 0x0000000c688a5800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=28419, stack(0x000000016f4e8000,0x000000016f8eb000)] - 0x0000000c688a5e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=27907, stack(0x000000016f8f4000,0x000000016faf7000)] - 0x0000000c688a6400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=25347, stack(0x000000016fb00000,0x000000016fd03000)] - 0x0000000c688a6a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=27651, stack(0x000000016fd0c000,0x000000017010f000)] - 0x0000000c688a7000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27395, stack(0x0000000170118000,0x000000017051b000)] - 0x0000000c6890a400 JavaThread "Notification Thread" daemon [_thread_blocked, id=42499, stack(0x0000000170f60000,0x0000000171363000)] - 0x0000000c6862c600 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=65283, stack(0x0000000173e68000,0x000000017426b000)] - 0x0000000c66c26a00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=47107, stack(0x0000000174cb0000,0x00000001750b3000)] - 0x0000000c687c5800 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=63747, stack(0x00000001750bc000,0x00000001754bf000)] - 0x0000000c687c5200 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=63235, stack(0x00000001754c8000,0x00000001758cb000)] - 0x0000000c687c4c00 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=62979, stack(0x00000001758d4000,0x0000000175cd7000)] - 0x0000000c687c4600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=47875, stack(0x0000000175ce0000,0x00000001760e3000)] - 0x0000000c687c5e00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=48131, stack(0x00000001760ec000,0x00000001764ef000)] - 0x0000000c66cbc000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=62211, stack(0x00000001764f8000,0x00000001768fb000)] - 0x0000000c66cbc600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=48899, stack(0x0000000176904000,0x0000000176d07000)] - 0x0000000c66cbcc00 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=61699, stack(0x0000000176d10000,0x0000000177113000)] - 0x0000000c66cbd200 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=49411, stack(0x000000017711c000,0x000000017751f000)] - 0x0000000c66cbd800 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=61187, stack(0x0000000177528000,0x000000017792b000)] - 0x0000000c66cbde00 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=49667, stack(0x0000000177934000,0x0000000177d37000)] - 0x0000000c66cbe400 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=60675, stack(0x0000000177d40000,0x0000000178143000)] - 0x0000000c66cbea00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=60419, stack(0x000000017814c000,0x000000017854f000)] - 0x0000000c66ccd200 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=50435, stack(0x0000000178558000,0x000000017895b000)] - 0x0000000c66ccf000 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=59651, stack(0x0000000178964000,0x0000000178d67000)] - 0x0000000c66ccf600 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=59395, stack(0x0000000178d70000,0x0000000179173000)] - 0x0000000c66d1c600 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=50947, stack(0x000000017917c000,0x000000017957f000)] - 0x0000000c66d1cc00 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=51459, stack(0x0000000179588000,0x000000017998b000)] - 0x0000000c66d1d800 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=58627, stack(0x0000000179994000,0x0000000179d97000)] - 0x0000000c66d1de00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=51971, stack(0x0000000179da0000,0x000000017a1a3000)] - 0x0000000c66d1e400 JavaThread "task-abort-timer" daemon [_thread_blocked, id=52227, stack(0x000000017a1ac000,0x000000017a5af000)] - 0x0000000c66d1f000 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=57859, stack(0x000000017a5b8000,0x000000017a9bb000)] - 0x0000000c66d1f600 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=52483, stack(0x000000017a9c4000,0x000000017adc7000)] - 0x0000000c66d68c00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=52995, stack(0x000000017add0000,0x000000017b1d3000)] - 0x0000000c66d69200 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=53507, stack(0x000000017b1dc000,0x000000017b5df000)] - 0x0000000c66d69e00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=54019, stack(0x000000017b5e8000,0x000000017b9eb000)] - 0x0000000c66d6aa00 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=54531, stack(0x000000017b9f4000,0x000000017bdf7000)] - 0x0000000c66d6b000 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=55043, stack(0x000000017be00000,0x000000017c203000)] - 0x0000000c66d6b600 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=55299, stack(0x000000017c20c000,0x000000017c60f000)] - 0x0000000c66dd8000 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=55555, stack(0x000000017c618000,0x000000017ca1b000)] - 0x0000000c66de4600 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=55811, stack(0x000000017ca24000,0x000000017ce27000)] - 0x0000000c66de5e00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=56323, stack(0x000000017ce30000,0x000000017d233000)] - 0x0000000c66ec1800 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=65795, stack(0x000000017d648000,0x000000017da4b000)] - 0x0000000c66530c00 JavaThread "process reaper" daemon [_thread_blocked, id=34067, stack(0x0000000171578000,0x00000001715af000)] - 0x0000000c67a99800 JavaThread "Executor task launch worker for task 8.0 in stage 127.0 (TID 609)" daemon [_thread_blocked, id=43331, stack(0x0000000170b48000,0x0000000170f4b000)] - 0x0000000c67a99e00 JavaThread "Executor task launch worker for task 5.0 in stage 267.0 (TID 1087)" daemon [_thread_blocked, id=42787, stack(0x000000017da54000,0x000000017de57000)] -=>0x0000000c67a9a400 JavaThread "Executor task launch worker for task 0.0 in stage 268.0 (TID 1092)" daemon [_thread_in_vm, id=29447, stack(0x000000017de60000,0x000000017e263000)] - 0x0000000c67a9aa00 JavaThread "Executor task launch worker for task 7.0 in stage 267.0 (TID 1089)" daemon [_thread_blocked, id=86275, stack(0x000000017e26c000,0x000000017e66f000)] - 0x0000000c67a9b000 JavaThread "Executor task launch worker for task 9.0 in stage 267.0 (TID 1091)" daemon [_thread_blocked, id=66307, stack(0x000000017e678000,0x000000017ea7b000)] - 0x0000000c676b0600 JavaThread "process reaper" daemon [_thread_blocked, id=67075, stack(0x000000016ec44000,0x000000016ec7b000)] - 0x0000000c676b2400 JavaThread "process reaper" daemon [_thread_blocked, id=67587, stack(0x000000016ec84000,0x000000016ecbb000)] - 0x0000000c67aab600 JavaThread "process reaper" daemon [_thread_blocked, id=84995, stack(0x00000001715b8000,0x00000001715ef000)] - 0x0000000c67aaaa00 JavaThread "process reaper" daemon [_thread_blocked, id=84483, stack(0x00000001715f8000,0x000000017162f000)] - 0x0000000c67130600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=68403, stack(0x000000017ea84000,0x000000017ee87000)] - 0x0000000c67130c00 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=83983, stack(0x000000017ee90000,0x000000017f293000)] - 0x0000000c67131200 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=66599, stack(0x000000017f29c000,0x000000017f69f000)] - 0x0000000c67131800 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=85519, stack(0x000000017f6a8000,0x000000017faab000)] - 0x0000000c65d64600 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=66875, stack(0x000000017fab4000,0x000000017feb7000)] - 0x0000000c65d64c00 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=68635, stack(0x0000000300004000,0x0000000300407000)] - 0x0000000c65d65200 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=68867, stack(0x0000000300410000,0x0000000300813000)] - 0x0000000c65d65800 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=82691, stack(0x000000030081c000,0x0000000300c1f000)] - 0x0000000c65d65e00 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=82179, stack(0x0000000300c28000,0x000000030102b000)] - 0x0000000c65e5a400 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=69379, stack(0x0000000301034000,0x0000000301437000)] - 0x0000000c65e59e00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=81923, stack(0x0000000301440000,0x0000000301843000)] - 0x0000000c65e5aa00 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=81667, stack(0x000000030184c000,0x0000000301c4f000)] - 0x0000000c65e5b000 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=70403, stack(0x0000000301c58000,0x000000030205b000)] - 0x0000000c65e5b600 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=81155, stack(0x0000000302064000,0x0000000302467000)] - 0x0000000c64688000 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=65031, stack(0x00000003034d0000,0x00000003038d3000)] - 0x0000000c6862f000 JavaThread "block-manager-storage-async-thread-pool-6" daemon [_thread_blocked, id=77571, stack(0x000000030410c000,0x000000030450f000)] - 0x0000000c677e6a00 JavaThread "block-manager-storage-async-thread-pool-7" daemon [_thread_blocked, id=73731, stack(0x0000000304518000,0x000000030491b000)] - 0x0000000c677e4000 JavaThread "block-manager-storage-async-thread-pool-8" daemon [_thread_blocked, id=77059, stack(0x0000000304924000,0x0000000304d27000)] - 0x0000000c677e4c00 JavaThread "block-manager-ask-thread-pool-4" daemon [_thread_blocked, id=74499, stack(0x0000000304d30000,0x0000000305133000)] - 0x0000000c67132a00 JavaThread "block-manager-ask-thread-pool-5" daemon [_thread_blocked, id=76547, stack(0x000000030513c000,0x000000030553f000)] - 0x0000000c654ad200 JavaThread "QueryStageCreator-1" daemon [_thread_blocked, id=72203, stack(0x0000000170524000,0x0000000170927000)] - 0x0000000c654ad800 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=79387, stack(0x0000000170930000,0x0000000170b33000)] - 0x0000000c654ade00 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=72467, stack(0x0000000174274000,0x0000000174477000)] - 0x0000000c654aea00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=45843, stack(0x0000000302eac000,0x00000003030af000)] - 0x0000000c654af000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=27155, stack(0x00000003032c4000,0x00000003034c7000)] - 0x0000000c654af600 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=78867, stack(0x00000003038dc000,0x0000000303adf000)] - 0x0000000c688a7600 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=26899, stack(0x0000000303ae8000,0x0000000303ceb000)] - 0x0000000c677e5800 JavaThread "C2 CompilerThread4" daemon [_thread_blocked, id=72975, stack(0x0000000303cf4000,0x0000000303ef7000)] - 0x0000000c677e5e00 JavaThread "C2 CompilerThread5" daemon [_thread_blocked, id=73223, stack(0x0000000303f00000,0x0000000304103000)] - 0x0000000c677e7000 JavaThread "QueryStageCreator-2" daemon [_thread_blocked, id=75779, stack(0x0000000306190000,0x0000000306593000)] - 0x0000000c677e7600 JavaThread "Timer-1" [_thread_blocked, id=65555, stack(0x000000017d23c000,0x000000017d63f000)] - 0x0000000c67aaa400 JavaThread "QueryStageCreator-3" daemon [_thread_blocked, id=87315, stack(0x000000030659c000,0x000000030699f000)] - 0x0000000c6890b600 JavaThread "block-manager-storage-async-thread-pool-9" daemon [_thread_blocked, id=87571, stack(0x00000003069a8000,0x0000000306dab000)] - 0x0000000c68908000 JavaThread "block-manager-storage-async-thread-pool-10" daemon [_thread_blocked, id=88067, stack(0x0000000306db4000,0x00000003071b7000)] - 0x0000000c65e59200 JavaThread "block-manager-storage-async-thread-pool-11" daemon [_thread_blocked, id=88579, stack(0x00000003071c0000,0x00000003075c3000)] - 0x0000000c646f8c00 JavaThread "block-manager-storage-async-thread-pool-12" daemon [_thread_blocked, id=130563, stack(0x00000003075cc000,0x00000003079cf000)] - 0x0000000c646f8600 JavaThread "block-manager-ask-thread-pool-6" daemon [_thread_blocked, id=130307, stack(0x00000003079d8000,0x0000000307ddb000)] - 0x0000000c640f0000 JavaThread "block-manager-ask-thread-pool-7" daemon [_thread_blocked, id=89347, stack(0x0000000307de4000,0x00000003081e7000)] - 0x0000000c640f0600 JavaThread "block-manager-storage-async-thread-pool-13" daemon [_thread_blocked, id=129539, stack(0x00000003081f0000,0x00000003085f3000)] - 0x0000000c640f0c00 JavaThread "block-manager-storage-async-thread-pool-14" daemon [_thread_blocked, id=129283, stack(0x00000003085fc000,0x00000003089ff000)] - 0x0000000c640f1200 JavaThread "block-manager-storage-async-thread-pool-15" daemon [_thread_blocked, id=90115, stack(0x0000000308a08000,0x0000000308e0b000)] - 0x0000000c640f1800 JavaThread "block-manager-storage-async-thread-pool-16" daemon [_thread_blocked, id=129027, stack(0x0000000308e14000,0x0000000309217000)] - 0x0000000c640f1e00 JavaThread "block-manager-storage-async-thread-pool-17" daemon [_thread_blocked, id=128771, stack(0x0000000309220000,0x0000000309623000)] - 0x0000000c640f2400 JavaThread "block-manager-ask-thread-pool-8" daemon [_thread_blocked, id=128515, stack(0x000000030962c000,0x0000000309a2f000)] - 0x0000000c640f2a00 JavaThread "block-manager-ask-thread-pool-9" daemon [_thread_blocked, id=128003, stack(0x0000000309a38000,0x0000000309e3b000)] - 0x0000000c640f3000 JavaThread "block-manager-ask-thread-pool-10" daemon [_thread_blocked, id=91139, stack(0x0000000309e44000,0x000000030a247000)] - 0x0000000c640f3600 JavaThread "block-manager-ask-thread-pool-11" daemon [_thread_blocked, id=127491, stack(0x000000030a250000,0x000000030a653000)] - 0x0000000c667b4000 JavaThread "block-manager-storage-async-thread-pool-18" daemon [_thread_blocked, id=126979, stack(0x000000030a65c000,0x000000030aa5f000)] - 0x0000000c667b4600 JavaThread "block-manager-storage-async-thread-pool-19" daemon [_thread_blocked, id=91907, stack(0x000000030aa68000,0x000000030ae6b000)] - 0x0000000c667b4c00 JavaThread "block-manager-storage-async-thread-pool-20" daemon [_thread_blocked, id=92419, stack(0x000000030ae74000,0x000000030b277000)] - 0x0000000c667b5200 JavaThread "block-manager-ask-thread-pool-12" daemon [_thread_blocked, id=92931, stack(0x000000030b280000,0x000000030b683000)] - 0x0000000c667b5800 JavaThread "block-manager-ask-thread-pool-13" daemon [_thread_blocked, id=93443, stack(0x000000030b68c000,0x000000030ba8f000)] - 0x0000000c667b5e00 JavaThread "block-manager-storage-async-thread-pool-21" daemon [_thread_blocked, id=126723, stack(0x000000030ba98000,0x000000030be9b000)] - 0x0000000c667b6400 JavaThread "block-manager-storage-async-thread-pool-22" daemon [_thread_blocked, id=126467, stack(0x000000030bea4000,0x000000030c2a7000)] - 0x0000000c667b6a00 JavaThread "block-manager-storage-async-thread-pool-23" daemon [_thread_blocked, id=126211, stack(0x000000030c2b0000,0x000000030c6b3000)] - 0x0000000c667b7000 JavaThread "block-manager-ask-thread-pool-14" daemon [_thread_blocked, id=125699, stack(0x000000030c6bc000,0x000000030cabf000)] - 0x0000000c667b7600 JavaThread "block-manager-ask-thread-pool-15" daemon [_thread_blocked, id=125443, stack(0x000000030cac8000,0x000000030cecb000)] - 0x0000000c6557c000 JavaThread "block-manager-storage-async-thread-pool-24" daemon [_thread_blocked, id=125187, stack(0x000000030ced4000,0x000000030d2d7000)] - 0x0000000c6557c600 JavaThread "block-manager-storage-async-thread-pool-25" daemon [_thread_blocked, id=95235, stack(0x000000030d2e0000,0x000000030d6e3000)] - 0x0000000c6557cc00 JavaThread "block-manager-storage-async-thread-pool-26" daemon [_thread_blocked, id=95491, stack(0x000000030d6ec000,0x000000030daef000)] - 0x0000000c6557d200 JavaThread "block-manager-ask-thread-pool-16" daemon [_thread_blocked, id=96003, stack(0x000000030daf8000,0x000000030defb000)] - 0x0000000c6557d800 JavaThread "block-manager-ask-thread-pool-17" daemon [_thread_blocked, id=96515, stack(0x000000030df04000,0x000000030e307000)] - 0x0000000c6557de00 JavaThread "block-manager-storage-async-thread-pool-27" daemon [_thread_blocked, id=124419, stack(0x000000030e310000,0x000000030e713000)] - 0x0000000c6557e400 JavaThread "block-manager-storage-async-thread-pool-28" daemon [_thread_blocked, id=96771, stack(0x000000030e71c000,0x000000030eb1f000)] - 0x0000000c6557ea00 JavaThread "block-manager-storage-async-thread-pool-29" daemon [_thread_blocked, id=97283, stack(0x000000030eb28000,0x000000030ef2b000)] - 0x0000000c6557f000 JavaThread "block-manager-ask-thread-pool-18" daemon [_thread_blocked, id=123907, stack(0x000000030ef34000,0x000000030f337000)] - 0x0000000c6557f600 JavaThread "block-manager-ask-thread-pool-19" daemon [_thread_blocked, id=98051, stack(0x000000030f340000,0x000000030f743000)] - 0x0000000c640fc000 JavaThread "block-manager-storage-async-thread-pool-30" daemon [_thread_blocked, id=98307, stack(0x000000030f74c000,0x000000030fb4f000)] - 0x0000000c640fc600 JavaThread "block-manager-storage-async-thread-pool-31" daemon [_thread_blocked, id=98563, stack(0x000000030fb58000,0x000000030ff5b000)] - 0x0000000c640fcc00 JavaThread "block-manager-storage-async-thread-pool-32" daemon [_thread_blocked, id=99075, stack(0x000000030ff64000,0x0000000310367000)] - 0x0000000c640fd200 JavaThread "block-manager-ask-thread-pool-20" daemon [_thread_blocked, id=99587, stack(0x0000000310370000,0x0000000310773000)] - 0x0000000c640fd800 JavaThread "block-manager-ask-thread-pool-21" daemon [_thread_blocked, id=100099, stack(0x000000031077c000,0x0000000310b7f000)] - 0x0000000c640ff000 JavaThread "QueryStageCreator-4" daemon [_thread_blocked, id=122883, stack(0x0000000310b88000,0x0000000310f8b000)] - 0x0000000c655bd200 JavaThread "QueryStageCreator-5" daemon [_thread_blocked, id=122371, stack(0x0000000310f94000,0x0000000311397000)] - 0x0000000c655be400 JavaThread "QueryStageCreator-6" daemon [_thread_blocked, id=121875, stack(0x00000003113a0000,0x00000003117a3000)] - 0x0000000c655bea00 JavaThread "block-manager-storage-async-thread-pool-33" daemon [_thread_blocked, id=121619, stack(0x00000003117ac000,0x0000000311baf000)] - 0x0000000c655bf000 JavaThread "block-manager-storage-async-thread-pool-34" daemon [_thread_blocked, id=121091, stack(0x0000000311bb8000,0x0000000311fbb000)] - 0x0000000c655bf600 JavaThread "block-manager-storage-async-thread-pool-35" daemon [_thread_blocked, id=100611, stack(0x0000000311fc4000,0x00000003123c7000)] - 0x0000000c66de6a00 JavaThread "block-manager-storage-async-thread-pool-36" daemon [_thread_blocked, id=120323, stack(0x00000003123d0000,0x00000003127d3000)] - 0x0000000c655e4000 JavaThread "block-manager-ask-thread-pool-22" daemon [_thread_blocked, id=119811, stack(0x00000003127dc000,0x0000000312bdf000)] - 0x0000000c655e4600 JavaThread "block-manager-storage-async-thread-pool-37" daemon [_thread_blocked, id=100867, stack(0x0000000312be8000,0x0000000312feb000)] - 0x0000000c655e4c00 JavaThread "block-manager-ask-thread-pool-23" daemon [_thread_blocked, id=101123, stack(0x0000000312ff4000,0x00000003133f7000)] - 0x0000000c655e5200 JavaThread "block-manager-storage-async-thread-pool-38" daemon [_thread_blocked, id=101635, stack(0x0000000313400000,0x0000000313803000)] - 0x0000000c655e5800 JavaThread "block-manager-ask-thread-pool-24" daemon [_thread_blocked, id=119043, stack(0x000000031380c000,0x0000000313c0f000)] - 0x0000000c655e5e00 JavaThread "block-manager-ask-thread-pool-25" daemon [_thread_blocked, id=102403, stack(0x0000000313c18000,0x000000031401b000)] - 0x0000000c655e6400 JavaThread "block-manager-storage-async-thread-pool-39" daemon [_thread_blocked, id=102659, stack(0x0000000314024000,0x0000000314427000)] - 0x0000000c655e6a00 JavaThread "block-manager-storage-async-thread-pool-40" daemon [_thread_blocked, id=118531, stack(0x0000000314430000,0x0000000314833000)] - 0x0000000c655e7000 JavaThread "block-manager-storage-async-thread-pool-41" daemon [_thread_blocked, id=118019, stack(0x000000031483c000,0x0000000314c3f000)] - 0x0000000c655e7600 JavaThread "block-manager-ask-thread-pool-26" daemon [_thread_blocked, id=117763, stack(0x0000000314c48000,0x000000031504b000)] - 0x0000000c64724000 JavaThread "block-manager-ask-thread-pool-27" daemon [_thread_blocked, id=103427, stack(0x0000000315054000,0x0000000315457000)] - 0x0000000c64724600 JavaThread "block-manager-storage-async-thread-pool-42" daemon [_thread_blocked, id=103683, stack(0x0000000315460000,0x0000000315863000)] - 0x0000000c64724c00 JavaThread "block-manager-storage-async-thread-pool-43" daemon [_thread_blocked, id=103939, stack(0x000000031586c000,0x0000000315c6f000)] - 0x0000000c64725200 JavaThread "block-manager-storage-async-thread-pool-44" daemon [_thread_blocked, id=116739, stack(0x0000000315c78000,0x000000031607b000)] - 0x0000000c64725800 JavaThread "block-manager-storage-async-thread-pool-45" daemon [_thread_blocked, id=104707, stack(0x0000000316084000,0x0000000316487000)] - 0x0000000c64725e00 JavaThread "block-manager-ask-thread-pool-28" daemon [_thread_blocked, id=104963, stack(0x0000000316490000,0x0000000316893000)] - 0x0000000c64726400 JavaThread "block-manager-storage-async-thread-pool-46" daemon [_thread_blocked, id=105219, stack(0x000000031689c000,0x0000000316c9f000)] - 0x0000000c64726a00 JavaThread "block-manager-ask-thread-pool-29" daemon [_thread_blocked, id=105731, stack(0x0000000316ca8000,0x00000003170ab000)] - 0x0000000c64727000 JavaThread "block-manager-storage-async-thread-pool-47" daemon [_thread_blocked, id=106243, stack(0x00000003170b4000,0x00000003174b7000)] - 0x0000000c64727600 JavaThread "block-manager-ask-thread-pool-30" daemon [_thread_blocked, id=106755, stack(0x00000003174c0000,0x00000003178c3000)] - 0x0000000c6412c000 JavaThread "block-manager-ask-thread-pool-31" daemon [_thread_blocked, id=107011, stack(0x00000003178cc000,0x0000000317ccf000)] - 0x0000000c6412c600 JavaThread "block-manager-storage-async-thread-pool-48" daemon [_thread_blocked, id=107267, stack(0x0000000317cd8000,0x00000003180db000)] - 0x0000000c6412cc00 JavaThread "block-manager-storage-async-thread-pool-49" daemon [_thread_blocked, id=115203, stack(0x00000003180e4000,0x00000003184e7000)] - 0x0000000c6412d200 JavaThread "block-manager-storage-async-thread-pool-50" daemon [_thread_blocked, id=107779, stack(0x00000003184f0000,0x00000003188f3000)] - 0x0000000c6412d800 JavaThread "block-manager-ask-thread-pool-32" daemon [_thread_blocked, id=114947, stack(0x00000003188fc000,0x0000000318cff000)] - 0x0000000c6412e400 JavaThread "block-manager-ask-thread-pool-33" daemon [_thread_blocked, id=114691, stack(0x0000000318d08000,0x000000031910b000)] - 0x0000000c6412de00 JavaThread "block-manager-storage-async-thread-pool-51" daemon [_thread_blocked, id=108547, stack(0x0000000319114000,0x0000000319517000)] - 0x0000000c6412ea00 JavaThread "block-manager-storage-async-thread-pool-52" daemon [_thread_blocked, id=114179, stack(0x0000000319520000,0x0000000319923000)] - 0x0000000c6412f000 JavaThread "block-manager-storage-async-thread-pool-53" daemon [_thread_blocked, id=113667, stack(0x000000031992c000,0x0000000319d2f000)] - 0x0000000c64134600 JavaThread "block-manager-ask-thread-pool-34" daemon [_thread_blocked, id=113155, stack(0x0000000319d38000,0x000000031a13b000)] - 0x0000000c64134000 JavaThread "block-manager-ask-thread-pool-35" daemon [_thread_blocked, id=109315, stack(0x000000031a144000,0x000000031a547000)] - 0x0000000c64134c00 JavaThread "block-manager-storage-async-thread-pool-54" daemon [_thread_blocked, id=109571, stack(0x000000031a550000,0x000000031a953000)] - 0x0000000c64135200 JavaThread "block-manager-storage-async-thread-pool-55" daemon [_thread_blocked, id=112387, stack(0x000000031a95c000,0x000000031ad5f000)] - 0x0000000c64135800 JavaThread "block-manager-storage-async-thread-pool-56" daemon [_thread_blocked, id=109827, stack(0x000000031ad68000,0x000000031b16b000)] - 0x0000000c64135e00 JavaThread "block-manager-ask-thread-pool-36" daemon [_thread_blocked, id=111619, stack(0x000000031b174000,0x000000031b577000)] - 0x0000000c64136400 JavaThread "block-manager-ask-thread-pool-37" daemon [_thread_blocked, id=111363, stack(0x000000031b580000,0x000000031b983000)] - 0x0000000c64136a00 JavaThread "block-manager-storage-async-thread-pool-57" daemon [_thread_blocked, id=110851, stack(0x000000031b98c000,0x000000031bd8f000)] - 0x0000000c64137000 JavaThread "block-manager-storage-async-thread-pool-58" daemon [_thread_blocked, id=110339, stack(0x000000031bd98000,0x000000031c19b000)] - 0x0000000c64137600 JavaThread "block-manager-storage-async-thread-pool-59" daemon [_thread_blocked, id=131331, stack(0x000000031c1a4000,0x000000031c5a7000)] - 0x0000000c655ec600 JavaThread "block-manager-ask-thread-pool-38" daemon [_thread_blocked, id=174083, stack(0x000000031c5b0000,0x000000031c9b3000)] - 0x0000000c655ec000 JavaThread "block-manager-ask-thread-pool-39" daemon [_thread_blocked, id=131843, stack(0x000000031c9bc000,0x000000031cdbf000)] - 0x0000000c655ecc00 JavaThread "block-manager-storage-async-thread-pool-60" daemon [_thread_blocked, id=132355, stack(0x000000031cdc8000,0x000000031d1cb000)] - 0x0000000c655ed200 JavaThread "block-manager-storage-async-thread-pool-61" daemon [_thread_blocked, id=132611, stack(0x000000031d1d4000,0x000000031d5d7000)] - 0x0000000c655ed800 JavaThread "block-manager-storage-async-thread-pool-62" daemon [_thread_blocked, id=132867, stack(0x000000031d5e0000,0x000000031d9e3000)] - 0x0000000c655ede00 JavaThread "block-manager-storage-async-thread-pool-63" daemon [_thread_blocked, id=173315, stack(0x000000031d9ec000,0x000000031ddef000)] - 0x0000000c655ee400 JavaThread "block-manager-ask-thread-pool-40" daemon [_thread_blocked, id=133635, stack(0x000000031ddf8000,0x000000031e1fb000)] - 0x0000000c655eea00 JavaThread "block-manager-storage-async-thread-pool-64" daemon [_thread_blocked, id=134147, stack(0x000000031e204000,0x000000031e607000)] - 0x0000000c655ef000 JavaThread "block-manager-ask-thread-pool-41" daemon [_thread_blocked, id=172803, stack(0x000000031e610000,0x000000031ea13000)] - 0x0000000c655ef600 JavaThread "block-manager-storage-async-thread-pool-65" daemon [_thread_blocked, id=172291, stack(0x000000031ea1c000,0x000000031ee1f000)] - 0x0000000c655f0600 JavaThread "block-manager-ask-thread-pool-42" daemon [_thread_blocked, id=171779, stack(0x000000031ee28000,0x000000031f22b000)] - 0x0000000c655f0000 JavaThread "block-manager-ask-thread-pool-43" daemon [_thread_blocked, id=134403, stack(0x000000031f234000,0x000000031f637000)] - 0x0000000c655f1e00 JavaThread "QueryStageCreator-7" daemon [_thread_blocked, id=171011, stack(0x000000031f640000,0x000000031fa43000)] - 0x0000000c655f3600 JavaThread "QueryStageCreator-8" daemon [_thread_blocked, id=134915, stack(0x000000031fa4c000,0x000000031fe4f000)] - 0x0000000c64738600 JavaThread "QueryStageCreator-9" daemon [_thread_blocked, id=170531, stack(0x000000031fe58000,0x000000032025b000)] - 0x0000000c6473b600 JavaThread "block-manager-storage-async-thread-pool-66" daemon [_thread_blocked, id=135179, stack(0x0000000320264000,0x0000000320667000)] - 0x0000000c6473b000 JavaThread "block-manager-storage-async-thread-pool-67" daemon [_thread_blocked, id=169731, stack(0x0000000320670000,0x0000000320a73000)] - 0x0000000c655f2400 JavaThread "block-manager-storage-async-thread-pool-68" daemon [_thread_blocked, id=169219, stack(0x0000000320a7c000,0x0000000320e7f000)] - 0x0000000c6412f600 JavaThread "block-manager-ask-thread-pool-44" daemon [_thread_blocked, id=168707, stack(0x0000000320e88000,0x000000032128b000)] - 0x0000000c64754000 JavaThread "block-manager-ask-thread-pool-45" daemon [_thread_blocked, id=168451, stack(0x0000000321294000,0x0000000321697000)] - 0x0000000c64754600 JavaThread "block-manager-storage-async-thread-pool-69" daemon [_thread_blocked, id=167939, stack(0x00000003216a0000,0x0000000321aa3000)] - 0x0000000c64754c00 JavaThread "block-manager-storage-async-thread-pool-70" daemon [_thread_blocked, id=135939, stack(0x0000000321aac000,0x0000000321eaf000)] - 0x0000000c64755200 JavaThread "block-manager-storage-async-thread-pool-71" daemon [_thread_blocked, id=167683, stack(0x0000000321eb8000,0x00000003222bb000)] - 0x0000000c64755800 JavaThread "block-manager-ask-thread-pool-46" daemon [_thread_blocked, id=136451, stack(0x00000003222c4000,0x00000003226c7000)] - 0x0000000c64755e00 JavaThread "block-manager-ask-thread-pool-47" daemon [_thread_blocked, id=136707, stack(0x00000003226d0000,0x0000000322ad3000)] - 0x0000000c64756400 JavaThread "block-manager-storage-async-thread-pool-72" daemon [_thread_blocked, id=137219, stack(0x0000000322adc000,0x0000000322edf000)] - 0x0000000c64756a00 JavaThread "block-manager-storage-async-thread-pool-73" daemon [_thread_blocked, id=137475, stack(0x0000000322ee8000,0x00000003232eb000)] - 0x0000000c64757000 JavaThread "block-manager-storage-async-thread-pool-74" daemon [_thread_blocked, id=166403, stack(0x00000003232f4000,0x00000003236f7000)] - 0x0000000c64757600 JavaThread "block-manager-storage-async-thread-pool-75" daemon [_thread_blocked, id=166147, stack(0x0000000323700000,0x0000000323b03000)] - 0x0000000c64758000 JavaThread "block-manager-ask-thread-pool-48" daemon [_thread_blocked, id=165635, stack(0x0000000323b0c000,0x0000000323f0f000)] - 0x0000000c64758600 JavaThread "block-manager-ask-thread-pool-49" daemon [_thread_blocked, id=137987, stack(0x0000000323f18000,0x000000032431b000)] - 0x0000000c64758c00 JavaThread "block-manager-storage-async-thread-pool-76" daemon [_thread_blocked, id=138243, stack(0x0000000324324000,0x0000000324727000)] - 0x0000000c64759200 JavaThread "block-manager-storage-async-thread-pool-77" daemon [_thread_blocked, id=138755, stack(0x0000000324730000,0x0000000324b33000)] - 0x0000000c64759800 JavaThread "block-manager-ask-thread-pool-50" daemon [_thread_blocked, id=139267, stack(0x0000000324b3c000,0x0000000324f3f000)] - 0x0000000c64759e00 JavaThread "block-manager-ask-thread-pool-51" daemon [_thread_blocked, id=164867, stack(0x0000000324f48000,0x000000032534b000)] - 0x0000000c6475a400 JavaThread "block-manager-storage-async-thread-pool-78" daemon [_thread_blocked, id=164355, stack(0x0000000325354000,0x0000000325757000)] - 0x0000000c6475aa00 JavaThread "block-manager-storage-async-thread-pool-79" daemon [_thread_blocked, id=163843, stack(0x0000000325760000,0x0000000325b63000)] - 0x0000000c6475b000 JavaThread "block-manager-storage-async-thread-pool-80" daemon [_thread_blocked, id=140035, stack(0x0000000325b6c000,0x0000000325f6f000)] - 0x0000000c6475b600 JavaThread "block-manager-ask-thread-pool-52" daemon [_thread_blocked, id=140547, stack(0x0000000325f78000,0x000000032637b000)] - 0x0000000c65600000 JavaThread "block-manager-ask-thread-pool-53" daemon [_thread_blocked, id=140803, stack(0x0000000326384000,0x0000000326787000)] - 0x0000000c65600600 JavaThread "block-manager-storage-async-thread-pool-81" daemon [_thread_blocked, id=141315, stack(0x0000000326790000,0x0000000326b93000)] - 0x0000000c65601200 JavaThread "block-manager-storage-async-thread-pool-82" daemon [_thread_blocked, id=141571, stack(0x0000000326b9c000,0x0000000326f9f000)] - 0x0000000c65601800 JavaThread "block-manager-storage-async-thread-pool-83" daemon [_thread_blocked, id=163075, stack(0x0000000326fa8000,0x00000003273ab000)] - 0x0000000c65601e00 JavaThread "block-manager-ask-thread-pool-54" daemon [_thread_blocked, id=142339, stack(0x00000003273b4000,0x00000003277b7000)] - 0x0000000c65602400 JavaThread "block-manager-ask-thread-pool-55" daemon [_thread_blocked, id=162563, stack(0x00000003277c0000,0x0000000327bc3000)] - 0x0000000c65602a00 JavaThread "block-manager-storage-async-thread-pool-84" daemon [_thread_blocked, id=162051, stack(0x0000000327bcc000,0x0000000327fcf000)] - 0x0000000c65603000 JavaThread "block-manager-storage-async-thread-pool-85" daemon [_thread_blocked, id=142851, stack(0x0000000327fd8000,0x00000003283db000)] - 0x0000000c65603600 JavaThread "block-manager-storage-async-thread-pool-86" daemon [_thread_blocked, id=143363, stack(0x00000003283e4000,0x00000003287e7000)] - 0x0000000c65608000 JavaThread "block-manager-storage-async-thread-pool-87" daemon [_thread_blocked, id=161539, stack(0x00000003287f0000,0x0000000328bf3000)] - 0x0000000c65608600 JavaThread "block-manager-ask-thread-pool-56" daemon [_thread_blocked, id=161283, stack(0x0000000328bfc000,0x0000000328fff000)] - 0x0000000c65608c00 JavaThread "block-manager-storage-async-thread-pool-88" daemon [_thread_blocked, id=161027, stack(0x0000000329008000,0x000000032940b000)] - 0x0000000c65609200 JavaThread "block-manager-storage-async-thread-pool-89" daemon [_thread_blocked, id=160771, stack(0x0000000329414000,0x0000000329817000)] - 0x0000000c65609800 JavaThread "block-manager-ask-thread-pool-57" daemon [_thread_blocked, id=160515, stack(0x0000000329820000,0x0000000329c23000)] - 0x0000000c65609e00 JavaThread "block-manager-storage-async-thread-pool-90" daemon [_thread_blocked, id=144899, stack(0x0000000329c2c000,0x000000032a02f000)] - 0x0000000c6560a400 JavaThread "block-manager-ask-thread-pool-58" daemon [_thread_blocked, id=145155, stack(0x000000032a038000,0x000000032a43b000)] - 0x0000000c6560aa00 JavaThread "block-manager-ask-thread-pool-59" daemon [_thread_blocked, id=160003, stack(0x000000032a444000,0x000000032a847000)] - 0x0000000c6560b000 JavaThread "block-manager-storage-async-thread-pool-91" daemon [_thread_blocked, id=159747, stack(0x000000032a850000,0x000000032ac53000)] - 0x0000000c6560b600 JavaThread "block-manager-storage-async-thread-pool-92" daemon [_thread_blocked, id=159235, stack(0x000000032ac5c000,0x000000032b05f000)] - 0x0000000c64170000 JavaThread "block-manager-ask-thread-pool-60" daemon [_thread_blocked, id=158979, stack(0x000000032b068000,0x000000032b46b000)] - 0x0000000c64170600 JavaThread "block-manager-ask-thread-pool-61" daemon [_thread_blocked, id=146435, stack(0x000000032b474000,0x000000032b877000)] - 0x0000000c64170c00 JavaThread "block-manager-storage-async-thread-pool-93" daemon [_thread_blocked, id=158467, stack(0x000000032b880000,0x000000032bc83000)] - 0x0000000c64171200 JavaThread "block-manager-storage-async-thread-pool-94" daemon [_thread_blocked, id=146691, stack(0x000000032bc8c000,0x000000032c08f000)] - 0x0000000c64171800 JavaThread "block-manager-storage-async-thread-pool-95" daemon [_thread_blocked, id=157955, stack(0x000000032c098000,0x000000032c49b000)] - 0x0000000c64171e00 JavaThread "block-manager-ask-thread-pool-62" daemon [_thread_blocked, id=147459, stack(0x000000032c4a4000,0x000000032c8a7000)] - 0x0000000c64172400 JavaThread "block-manager-ask-thread-pool-63" daemon [_thread_blocked, id=157699, stack(0x000000032c8b0000,0x000000032ccb3000)] - 0x0000000c64172a00 JavaThread "block-manager-storage-async-thread-pool-96" daemon [_thread_blocked, id=148227, stack(0x000000032ccbc000,0x000000032d0bf000)] - 0x0000000c64173000 JavaThread "block-manager-storage-async-thread-pool-97" daemon [_thread_blocked, id=157187, stack(0x000000032d0c8000,0x000000032d4cb000)] - 0x0000000c64173600 JavaThread "block-manager-storage-async-thread-pool-98" daemon [_thread_blocked, id=156675, stack(0x000000032d4d4000,0x000000032d8d7000)] - 0x0000000c64768000 JavaThread "block-manager-ask-thread-pool-64" daemon [_thread_blocked, id=148483, stack(0x000000032d8e0000,0x000000032dce3000)] - 0x0000000c64768600 JavaThread "block-manager-ask-thread-pool-65" daemon [_thread_blocked, id=155907, stack(0x000000032dcec000,0x000000032e0ef000)] - 0x0000000c64769200 JavaThread "QueryStageCreator-10" daemon [_thread_blocked, id=155395, stack(0x000000032e0f8000,0x000000032e4fb000)] - 0x0000000c64769800 JavaThread "QueryStageCreator-11" daemon [_thread_blocked, id=148739, stack(0x000000032e504000,0x000000032e907000)] - 0x0000000c641a0600 JavaThread "QueryStageCreator-12" daemon [_thread_blocked, id=154891, stack(0x000000032e910000,0x000000032ed13000)] - 0x0000000c641a1800 JavaThread "block-manager-storage-async-thread-pool-99" daemon [_thread_blocked, id=154659, stack(0x000000032ed1c000,0x000000032f11f000)] - 0x0000000c641a1e00 JavaThread "block-manager-ask-thread-pool-66" daemon [_thread_blocked, id=149251, stack(0x000000032f128000,0x000000032f52b000)] - 0x0000000c641a2400 JavaThread "block-manager-ask-thread-pool-67" daemon [_thread_blocked, id=153859, stack(0x000000032f534000,0x000000032f937000)] - 0x0000000c641a2a00 JavaThread "block-manager-ask-thread-pool-68" daemon [_thread_blocked, id=153347, stack(0x000000032f940000,0x000000032fd43000)] - 0x0000000c641a3000 JavaThread "block-manager-ask-thread-pool-69" daemon [_thread_blocked, id=152835, stack(0x000000032fd4c000,0x000000033014f000)] - 0x0000000c641a3600 JavaThread "block-manager-ask-thread-pool-70" daemon [_thread_blocked, id=152323, stack(0x0000000330158000,0x000000033055b000)] - 0x0000000c646fb600 JavaThread "block-manager-ask-thread-pool-71" daemon [_thread_blocked, id=151811, stack(0x0000000330564000,0x0000000330967000)] - 0x0000000c65dbde00 JavaThread "block-manager-ask-thread-pool-72" daemon [_thread_blocked, id=151555, stack(0x0000000330970000,0x0000000330d73000)] - 0x0000000c646e1200 JavaThread "block-manager-ask-thread-pool-73" daemon [_thread_blocked, id=150019, stack(0x0000000330d7c000,0x000000033117f000)] - 0x0000000c66531800 JavaThread "block-manager-ask-thread-pool-74" daemon [_thread_blocked, id=150787, stack(0x0000000331188000,0x000000033158b000)] - 0x0000000c66531200 JavaThread "block-manager-ask-thread-pool-75" daemon [_thread_blocked, id=150275, stack(0x0000000331594000,0x0000000331997000)] - 0x0000000c641a8000 JavaThread "block-manager-ask-thread-pool-76" daemon [_thread_blocked, id=174851, stack(0x00000003319a0000,0x0000000331da3000)] - 0x0000000c641a8600 JavaThread "block-manager-ask-thread-pool-77" daemon [_thread_blocked, id=261891, stack(0x0000000331dac000,0x00000003321af000)] - 0x0000000c641a8c00 JavaThread "block-manager-ask-thread-pool-78" daemon [_thread_blocked, id=261635, stack(0x00000003321b8000,0x00000003325bb000)] - 0x0000000c641a9200 JavaThread "block-manager-ask-thread-pool-79" daemon [_thread_blocked, id=261123, stack(0x00000003325c4000,0x00000003329c7000)] - 0x0000000c641a9800 JavaThread "block-manager-ask-thread-pool-80" daemon [_thread_blocked, id=260867, stack(0x00000003329d0000,0x0000000332dd3000)] - 0x0000000c641a9e00 JavaThread "block-manager-ask-thread-pool-81" daemon [_thread_blocked, id=176131, stack(0x0000000332ddc000,0x00000003331df000)] - 0x0000000c641aa400 JavaThread "block-manager-ask-thread-pool-82" daemon [_thread_blocked, id=260355, stack(0x00000003331e8000,0x00000003335eb000)] - 0x0000000c641aaa00 JavaThread "block-manager-ask-thread-pool-83" daemon [_thread_blocked, id=176387, stack(0x00000003335f4000,0x00000003339f7000)] - 0x0000000c641ab000 JavaThread "block-manager-ask-thread-pool-84" daemon [_thread_blocked, id=259587, stack(0x0000000333a00000,0x0000000333e03000)] - 0x0000000c641ab600 JavaThread "block-manager-ask-thread-pool-85" daemon [_thread_blocked, id=176643, stack(0x0000000333e0c000,0x000000033420f000)] - 0x0000000c6477c600 JavaThread "block-manager-ask-thread-pool-86" daemon [_thread_blocked, id=177155, stack(0x0000000334218000,0x000000033461b000)] - 0x0000000c6477d200 JavaThread "block-manager-ask-thread-pool-87" daemon [_thread_blocked, id=259075, stack(0x0000000334624000,0x0000000334a27000)] - 0x0000000c6477d800 JavaThread "QueryStageCreator-13" daemon [_thread_blocked, id=177923, stack(0x0000000334a30000,0x0000000334e33000)] - 0x0000000c6477e400 JavaThread "QueryStageCreator-14" daemon [_thread_blocked, id=258819, stack(0x0000000334e3c000,0x000000033523f000)] - 0x0000000c6477ea00 JavaThread "QueryStageCreator-15" daemon [_thread_blocked, id=258571, stack(0x0000000335248000,0x000000033564b000)] - 0x0000000c6477f600 JavaThread "block-manager-ask-thread-pool-88" daemon [_thread_blocked, id=179219, stack(0x0000000335654000,0x0000000335a57000)] - 0x0000000c64200000 JavaThread "block-manager-ask-thread-pool-89" daemon [_thread_blocked, id=179459, stack(0x0000000335a60000,0x0000000335e63000)] - 0x0000000c64200600 JavaThread "block-manager-ask-thread-pool-90" daemon [_thread_blocked, id=179715, stack(0x0000000335e6c000,0x000000033626f000)] - 0x0000000c64200c00 JavaThread "block-manager-ask-thread-pool-91" daemon [_thread_blocked, id=179971, stack(0x0000000336278000,0x000000033667b000)] - 0x0000000c64201200 JavaThread "block-manager-ask-thread-pool-92" daemon [_thread_blocked, id=180227, stack(0x0000000336684000,0x0000000336a87000)] - 0x0000000c64201800 JavaThread "block-manager-ask-thread-pool-93" daemon [_thread_blocked, id=257539, stack(0x0000000336a90000,0x0000000336e93000)] - 0x0000000c64201e00 JavaThread "block-manager-ask-thread-pool-94" daemon [_thread_blocked, id=180739, stack(0x0000000336e9c000,0x000000033729f000)] - 0x0000000c64202400 JavaThread "block-manager-ask-thread-pool-95" daemon [_thread_blocked, id=256771, stack(0x00000003372a8000,0x00000003376ab000)] - 0x0000000c64202a00 JavaThread "block-manager-ask-thread-pool-96" daemon [_thread_blocked, id=180995, stack(0x00000003376b4000,0x0000000337ab7000)] - 0x0000000c64203000 JavaThread "block-manager-ask-thread-pool-97" daemon [_thread_blocked, id=256003, stack(0x0000000337ac0000,0x0000000337ec3000)] - 0x0000000c64203600 JavaThread "block-manager-ask-thread-pool-98" daemon [_thread_blocked, id=255747, stack(0x0000000337ecc000,0x00000003382cf000)] - 0x0000000c64204000 JavaThread "block-manager-ask-thread-pool-99" daemon [_thread_blocked, id=181763, stack(0x00000003382d8000,0x00000003386db000)] - 0x0000000c64289e00 JavaThread "Executor task launch worker for task 8.0 in stage 267.0 (TID 1090)" daemon [_thread_blocked, id=255043, stack(0x00000003386e4000,0x0000000338ae7000)] - -Other Threads: - 0x0000000103401c20 VMThread "VM Thread" [stack: 0x000000016e194000,0x000000016e397000] [id=17923] - 0x0000000c68bf8600 WatcherThread [stack: 0x000000017136c000,0x000000017156f000] [id=42243] - 0x00000001033fb300 GCTaskThread "GC Thread#0" [stack: 0x000000016d758000,0x000000016d95b000] [id=14339] - 0x0000000c67438f00 GCTaskThread "GC Thread#1" [stack: 0x0000000171784000,0x0000000171987000] [id=34307] - 0x0000000c67439200 GCTaskThread "GC Thread#2" [stack: 0x0000000171990000,0x0000000171b93000] [id=34563] - 0x0000000c67439500 GCTaskThread "GC Thread#3" [stack: 0x0000000171b9c000,0x0000000171d9f000] [id=41475] - 0x0000000c67439800 GCTaskThread "GC Thread#4" [stack: 0x0000000171da8000,0x0000000171fab000] [id=35331] - 0x0000000c67439b00 GCTaskThread "GC Thread#5" [stack: 0x0000000171fb4000,0x00000001721b7000] [id=40963] - 0x0000000c67439e00 GCTaskThread "GC Thread#6" [stack: 0x00000001721c0000,0x00000001723c3000] [id=40451] - 0x0000000c6743a100 GCTaskThread "GC Thread#7" [stack: 0x00000001723cc000,0x00000001725cf000] [id=39939] - 0x0000000c6743a400 GCTaskThread "GC Thread#8" [stack: 0x00000001725d8000,0x00000001727db000] [id=39683] - 0x0000000c6743a700 GCTaskThread "GC Thread#9" [stack: 0x00000001727e4000,0x00000001729e7000] [id=39171] - 0x0000000c6743aa00 GCTaskThread "GC Thread#10" [stack: 0x00000001729f0000,0x0000000172bf3000] [id=36099] - 0x0000000c6743ad00 GCTaskThread "GC Thread#11" [stack: 0x0000000172bfc000,0x0000000172dff000] [id=36355] - 0x0000000c6743b000 GCTaskThread "GC Thread#12" [stack: 0x0000000172e08000,0x000000017300b000] [id=36611] - 0x0000000c6743b300 GCTaskThread "GC Thread#13" [stack: 0x0000000173014000,0x0000000173217000] [id=38147] - 0x0000000c6743b600 GCTaskThread "GC Thread#14" [stack: 0x0000000173220000,0x0000000173423000] [id=37635] - 0x0000000c6743b900 GCTaskThread "GC Thread#15" [stack: 0x000000017342c000,0x000000017362f000] [id=36867] - 0x0000000c6743bc00 GCTaskThread "GC Thread#16" [stack: 0x0000000173638000,0x000000017383b000] [id=43523] - 0x0000000c67450000 GCTaskThread "GC Thread#17" [stack: 0x0000000173844000,0x0000000173a47000] [id=44035] - 0x0000000c67450300 GCTaskThread "GC Thread#18" [stack: 0x0000000173a50000,0x0000000173c53000] [id=44547] - 0x0000000c67450600 GCTaskThread "GC Thread#19" [stack: 0x0000000173c5c000,0x0000000173e5f000] [id=45059] - 0x00000001033fbf00 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016d964000,0x000000016db67000] [id=13827] - 0x00000001033fc670 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016db70000,0x000000016dd73000] [id=13315] - 0x0000000c6755cf00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174480000,0x0000000174683000] [id=46083] - 0x0000000c6755d500 ConcurrentGCThread "G1 Conc#2" [stack: 0x000000017468c000,0x000000017488f000] [id=46339] - 0x0000000c6755d800 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000174898000,0x0000000174a9b000] [id=64259] - 0x0000000c6755db00 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000174aa4000,0x0000000174ca7000] [id=64003] - 0x00000001033fe810 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016dd7c000,0x000000016df7f000] [id=21507] - 0x00000001033fef10 ConcurrentGCThread "G1 Service" [stack: 0x000000016df88000,0x000000016e18b000] [id=16899] - -Threads with active compile tasks: - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x0000000700000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x000000c800000000-0x000000c800bc0000-0x000000c800bc0000), size 12320768, SharedBaseAddress: 0x000000c800000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x000000c801000000-0x000000c841000000, reserved size: 1073741824 -Narrow klass base: 0x000000c800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 954368K, used 430520K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 60 young (122880K), 4 survivors (8192K) - Metaspace used 170664K, committed 172096K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000700000000, 0x0000000700200000, 0x0000000700200000|100%|HS| |TAMS 0x0000000700200000, 0x0000000700200000| Complete -| 1|0x0000000700200000, 0x0000000700400000, 0x0000000700400000|100%|HS| |TAMS 0x0000000700400000, 0x0000000700400000| Complete -| 2|0x0000000700400000, 0x0000000700600000, 0x0000000700600000|100%|HC| |TAMS 0x0000000700600000, 0x0000000700600000| Complete -| 3|0x0000000700600000, 0x0000000700800000, 0x0000000700800000|100%|HS| |TAMS 0x0000000700800000, 0x0000000700800000| Complete -| 4|0x0000000700800000, 0x0000000700a00000, 0x0000000700a00000|100%|HC| |TAMS 0x0000000700a00000, 0x0000000700a00000| Complete -| 5|0x0000000700a00000, 0x0000000700c00000, 0x0000000700c00000|100%|HC| |TAMS 0x0000000700c00000, 0x0000000700c00000| Complete -| 6|0x0000000700c00000, 0x0000000700e00000, 0x0000000700e00000|100%|HC| |TAMS 0x0000000700e00000, 0x0000000700e00000| Complete -| 7|0x0000000700e00000, 0x0000000701000000, 0x0000000701000000|100%|HC| |TAMS 0x0000000701000000, 0x0000000701000000| Complete -| 8|0x0000000701000000, 0x0000000701200000, 0x0000000701200000|100%|HC| |TAMS 0x0000000701200000, 0x0000000701200000| Complete -| 9|0x0000000701200000, 0x0000000701400000, 0x0000000701400000|100%|HC| |TAMS 0x0000000701400000, 0x0000000701400000| Complete -| 10|0x0000000701400000, 0x0000000701600000, 0x0000000701600000|100%|HC| |TAMS 0x0000000701600000, 0x0000000701600000| Complete -| 11|0x0000000701600000, 0x0000000701800000, 0x0000000701800000|100%|HS| |TAMS 0x0000000701800000, 0x0000000701800000| Complete -| 12|0x0000000701800000, 0x0000000701a00000, 0x0000000701a00000|100%|HS| |TAMS 0x0000000701a00000, 0x0000000701a00000| Complete -| 13|0x0000000701a00000, 0x0000000701c00000, 0x0000000701c00000|100%|HC| |TAMS 0x0000000701c00000, 0x0000000701c00000| Complete -| 14|0x0000000701c00000, 0x0000000701e00000, 0x0000000701e00000|100%|HS| |TAMS 0x0000000701e00000, 0x0000000701e00000| Complete -| 15|0x0000000701e00000, 0x0000000702000000, 0x0000000702000000|100%|HC| |TAMS 0x0000000702000000, 0x0000000702000000| Complete -| 16|0x0000000702000000, 0x0000000702200000, 0x0000000702200000|100%|HS| |TAMS 0x0000000702200000, 0x0000000702200000| Complete -| 17|0x0000000702200000, 0x0000000702400000, 0x0000000702400000|100%|HS| |TAMS 0x0000000702400000, 0x0000000702400000| Complete -| 18|0x0000000702400000, 0x0000000702600000, 0x0000000702600000|100%|HC| |TAMS 0x0000000702600000, 0x0000000702600000| Complete -| 19|0x0000000702600000, 0x0000000702800000, 0x0000000702800000|100%| O| |TAMS 0x0000000702800000, 0x0000000702800000| Untracked -| 20|0x0000000702800000, 0x0000000702a00000, 0x0000000702a00000|100%| O| |TAMS 0x0000000702a00000, 0x0000000702a00000| Untracked -| 21|0x0000000702a00000, 0x0000000702c00000, 0x0000000702c00000|100%| O| |TAMS 0x0000000702c00000, 0x0000000702c00000| Untracked -| 22|0x0000000702c00000, 0x0000000702e00000, 0x0000000702e00000|100%| O| |TAMS 0x0000000702e00000, 0x0000000702e00000| Untracked -| 23|0x0000000702e00000, 0x0000000703000000, 0x0000000703000000|100%| O| |TAMS 0x0000000703000000, 0x0000000703000000| Untracked -| 24|0x0000000703000000, 0x0000000703200000, 0x0000000703200000|100%| O| |TAMS 0x0000000703200000, 0x0000000703200000| Untracked -| 25|0x0000000703200000, 0x0000000703400000, 0x0000000703400000|100%| O| |TAMS 0x0000000703400000, 0x0000000703400000| Untracked -| 26|0x0000000703400000, 0x0000000703600000, 0x0000000703600000|100%| O| |TAMS 0x0000000703600000, 0x0000000703600000| Untracked -| 27|0x0000000703600000, 0x0000000703800000, 0x0000000703800000|100%| O| |TAMS 0x0000000703800000, 0x0000000703800000| Untracked -| 28|0x0000000703800000, 0x0000000703a00000, 0x0000000703a00000|100%| O| |TAMS 0x0000000703a00000, 0x0000000703a00000| Untracked -| 29|0x0000000703a00000, 0x0000000703c00000, 0x0000000703c00000|100%| O| |TAMS 0x0000000703c00000, 0x0000000703c00000| Untracked -| 30|0x0000000703c00000, 0x0000000703e00000, 0x0000000703e00000|100%| O| |TAMS 0x0000000703e00000, 0x0000000703e00000| Untracked -| 31|0x0000000703e00000, 0x0000000704000000, 0x0000000704000000|100%| O| |TAMS 0x0000000704000000, 0x0000000704000000| Untracked -| 32|0x0000000704000000, 0x0000000704200000, 0x0000000704200000|100%| O| |TAMS 0x0000000704200000, 0x0000000704200000| Untracked -| 33|0x0000000704200000, 0x0000000704400000, 0x0000000704400000|100%| O| |TAMS 0x0000000704400000, 0x0000000704400000| Untracked -| 34|0x0000000704400000, 0x0000000704600000, 0x0000000704600000|100%| O| |TAMS 0x0000000704600000, 0x0000000704600000| Untracked -| 35|0x0000000704600000, 0x0000000704800000, 0x0000000704800000|100%| O| |TAMS 0x0000000704800000, 0x0000000704800000| Untracked -| 36|0x0000000704800000, 0x0000000704a00000, 0x0000000704a00000|100%| O| |TAMS 0x0000000704a00000, 0x0000000704a00000| Untracked -| 37|0x0000000704a00000, 0x0000000704c00000, 0x0000000704c00000|100%| O| |TAMS 0x0000000704c00000, 0x0000000704c00000| Untracked -| 38|0x0000000704c00000, 0x0000000704e00000, 0x0000000704e00000|100%| O| |TAMS 0x0000000704e00000, 0x0000000704e00000| Untracked -| 39|0x0000000704e00000, 0x0000000705000000, 0x0000000705000000|100%| O| |TAMS 0x0000000705000000, 0x0000000705000000| Untracked -| 40|0x0000000705000000, 0x0000000705200000, 0x0000000705200000|100%| O| |TAMS 0x0000000705200000, 0x0000000705200000| Untracked -| 41|0x0000000705200000, 0x0000000705400000, 0x0000000705400000|100%| O| |TAMS 0x0000000705200000, 0x0000000705400000| Untracked -| 42|0x0000000705400000, 0x0000000705600000, 0x0000000705600000|100%| O| |TAMS 0x0000000705600000, 0x0000000705600000| Untracked -| 43|0x0000000705600000, 0x0000000705800000, 0x0000000705800000|100%| O| |TAMS 0x0000000705800000, 0x0000000705800000| Untracked -| 44|0x0000000705800000, 0x0000000705a00000, 0x0000000705a00000|100%| O| |TAMS 0x0000000705a00000, 0x0000000705a00000| Untracked -| 45|0x0000000705a00000, 0x0000000705c00000, 0x0000000705c00000|100%|HS| |TAMS 0x0000000705a00000, 0x0000000705a00000| Complete -| 46|0x0000000705c00000, 0x0000000705e00000, 0x0000000705e00000|100%|HS| |TAMS 0x0000000705c00000, 0x0000000705c00000| Complete -| 47|0x0000000705e00000, 0x0000000706000000, 0x0000000706000000|100%| O| |TAMS 0x0000000706000000, 0x0000000706000000| Untracked -| 48|0x0000000706000000, 0x0000000706200000, 0x0000000706200000|100%| O| |TAMS 0x0000000706200000, 0x0000000706200000| Untracked -| 49|0x0000000706200000, 0x0000000706400000, 0x0000000706400000|100%|HS| |TAMS 0x0000000706200000, 0x0000000706200000| Complete -| 50|0x0000000706400000, 0x0000000706600000, 0x0000000706600000|100%|HS| |TAMS 0x0000000706400000, 0x0000000706400000| Complete -| 51|0x0000000706600000, 0x0000000706800000, 0x0000000706800000|100%|HS| |TAMS 0x0000000706600000, 0x0000000706600000| Complete -| 52|0x0000000706800000, 0x0000000706a00000, 0x0000000706a00000|100%|HS| |TAMS 0x0000000706800000, 0x0000000706800000| Complete -| 53|0x0000000706a00000, 0x0000000706c00000, 0x0000000706c00000|100%| O| |TAMS 0x0000000706c00000, 0x0000000706c00000| Untracked -| 54|0x0000000706c00000, 0x0000000706e00000, 0x0000000706e00000|100%| O| |TAMS 0x0000000706e00000, 0x0000000706e00000| Untracked -| 55|0x0000000706e00000, 0x0000000707000000, 0x0000000707000000|100%| O| |TAMS 0x0000000707000000, 0x0000000707000000| Untracked -| 56|0x0000000707000000, 0x0000000707200000, 0x0000000707200000|100%| O| |TAMS 0x0000000707200000, 0x0000000707200000| Untracked -| 57|0x0000000707200000, 0x0000000707400000, 0x0000000707400000|100%| O| |TAMS 0x0000000707400000, 0x0000000707400000| Untracked -| 58|0x0000000707400000, 0x0000000707600000, 0x0000000707600000|100%| O| |TAMS 0x0000000707600000, 0x0000000707600000| Untracked -| 59|0x0000000707600000, 0x0000000707800000, 0x0000000707800000|100%| O| |TAMS 0x0000000707800000, 0x0000000707800000| Untracked -| 60|0x0000000707800000, 0x0000000707a00000, 0x0000000707a00000|100%|HS| |TAMS 0x0000000707800000, 0x0000000707800000| Complete -| 61|0x0000000707a00000, 0x0000000707c00000, 0x0000000707c00000|100%|HS| |TAMS 0x0000000707a00000, 0x0000000707a00000| Complete -| 62|0x0000000707c00000, 0x0000000707e00000, 0x0000000707e00000|100%|HC| |TAMS 0x0000000707c00000, 0x0000000707c00000| Complete -| 63|0x0000000707e00000, 0x0000000708000000, 0x0000000708000000|100%|HC| |TAMS 0x0000000707e00000, 0x0000000707e00000| Complete -| 64|0x0000000708000000, 0x0000000708200000, 0x0000000708200000|100%|HS| |TAMS 0x0000000708000000, 0x0000000708000000| Complete -| 65|0x0000000708200000, 0x0000000708400000, 0x0000000708400000|100%|HS| |TAMS 0x0000000708200000, 0x0000000708200000| Complete -| 66|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| O| |TAMS 0x0000000708600000, 0x0000000708600000| Untracked -| 67|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| O| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked -| 68|0x0000000708800000, 0x0000000708a00000, 0x0000000708a00000|100%| O| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked -| 69|0x0000000708a00000, 0x0000000708c00000, 0x0000000708c00000|100%| O| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked -| 70|0x0000000708c00000, 0x0000000708e00000, 0x0000000708e00000|100%| O| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked -| 71|0x0000000708e00000, 0x0000000709000000, 0x0000000709000000|100%|HS| |TAMS 0x0000000708e00000, 0x0000000708e00000| Complete -| 72|0x0000000709000000, 0x0000000709200000, 0x0000000709200000|100%| O| |TAMS 0x00000007090e6e00, 0x0000000709200000| Untracked -| 73|0x0000000709200000, 0x0000000709400000, 0x0000000709400000|100%|HS| |TAMS 0x0000000709200000, 0x0000000709200000| Complete -| 74|0x0000000709400000, 0x0000000709600000, 0x0000000709600000|100%|HC| |TAMS 0x0000000709400000, 0x0000000709400000| Complete -| 75|0x0000000709600000, 0x0000000709800000, 0x0000000709800000|100%|HC| |TAMS 0x0000000709600000, 0x0000000709600000| Complete -| 76|0x0000000709800000, 0x0000000709a00000, 0x0000000709a00000|100%|HS| |TAMS 0x0000000709800000, 0x0000000709800000| Complete -| 77|0x0000000709a00000, 0x0000000709c00000, 0x0000000709c00000|100%|HS| |TAMS 0x0000000709a00000, 0x0000000709a00000| Complete -| 78|0x0000000709c00000, 0x0000000709e00000, 0x0000000709e00000|100%|HS| |TAMS 0x0000000709c00000, 0x0000000709c00000| Complete -| 79|0x0000000709e00000, 0x000000070a000000, 0x000000070a000000|100%|HS| |TAMS 0x0000000709e00000, 0x0000000709e00000| Complete -| 80|0x000000070a000000, 0x000000070a200000, 0x000000070a200000|100%|HS| |TAMS 0x000000070a000000, 0x000000070a000000| Complete -| 81|0x000000070a200000, 0x000000070a400000, 0x000000070a400000|100%|HS| |TAMS 0x000000070a200000, 0x000000070a200000| Complete -| 82|0x000000070a400000, 0x000000070a600000, 0x000000070a600000|100%|HS| |TAMS 0x000000070a400000, 0x000000070a400000| Complete -| 83|0x000000070a600000, 0x000000070a800000, 0x000000070a800000|100%|HS| |TAMS 0x000000070a600000, 0x000000070a600000| Complete -| 84|0x000000070a800000, 0x000000070aa00000, 0x000000070aa00000|100%|HS| |TAMS 0x000000070a800000, 0x000000070a800000| Complete -| 85|0x000000070aa00000, 0x000000070ac00000, 0x000000070ac00000|100%|HS| |TAMS 0x000000070aa00000, 0x000000070aa00000| Complete -| 86|0x000000070ac00000, 0x000000070ae00000, 0x000000070ae00000|100%|HS| |TAMS 0x000000070ac00000, 0x000000070ac00000| Complete -| 87|0x000000070ae00000, 0x000000070b000000, 0x000000070b000000|100%|HS| |TAMS 0x000000070ae00000, 0x000000070ae00000| Complete -| 88|0x000000070b000000, 0x000000070b200000, 0x000000070b200000|100%|HS| |TAMS 0x000000070b000000, 0x000000070b000000| Complete -| 89|0x000000070b200000, 0x000000070b400000, 0x000000070b400000|100%|HS| |TAMS 0x000000070b200000, 0x000000070b200000| Complete -| 90|0x000000070b400000, 0x000000070b600000, 0x000000070b600000|100%|HS| |TAMS 0x000000070b400000, 0x000000070b400000| Complete -| 91|0x000000070b600000, 0x000000070b800000, 0x000000070b800000|100%|HS| |TAMS 0x000000070b600000, 0x000000070b600000| Complete -| 92|0x000000070b800000, 0x000000070ba00000, 0x000000070ba00000|100%|HS| |TAMS 0x000000070b800000, 0x000000070b800000| Complete -| 93|0x000000070ba00000, 0x000000070bc00000, 0x000000070bc00000|100%|HS| |TAMS 0x000000070ba00000, 0x000000070ba00000| Complete -| 94|0x000000070bc00000, 0x000000070be00000, 0x000000070be00000|100%|HS| |TAMS 0x000000070bc00000, 0x000000070bc00000| Complete -| 95|0x000000070be00000, 0x000000070c000000, 0x000000070c000000|100%|HC| |TAMS 0x000000070be00000, 0x000000070be00000| Complete -| 96|0x000000070c000000, 0x000000070c200000, 0x000000070c200000|100%|HC| |TAMS 0x000000070c000000, 0x000000070c000000| Complete -| 97|0x000000070c200000, 0x000000070c400000, 0x000000070c400000|100%|HS| |TAMS 0x000000070c200000, 0x000000070c200000| Complete -| 98|0x000000070c400000, 0x000000070c600000, 0x000000070c600000|100%|HS| |TAMS 0x000000070c400000, 0x000000070c400000| Complete -| 99|0x000000070c600000, 0x000000070c800000, 0x000000070c800000|100%|HC| |TAMS 0x000000070c600000, 0x000000070c600000| Complete -| 100|0x000000070c800000, 0x000000070ca00000, 0x000000070ca00000|100%|HC| |TAMS 0x000000070c800000, 0x000000070c800000| Complete -| 101|0x000000070ca00000, 0x000000070cc00000, 0x000000070cc00000|100%|HS| |TAMS 0x000000070ca00000, 0x000000070ca00000| Complete -| 102|0x000000070cc00000, 0x000000070ce00000, 0x000000070ce00000|100%|HC| |TAMS 0x000000070cc00000, 0x000000070cc00000| Complete -| 103|0x000000070ce00000, 0x000000070d000000, 0x000000070d000000|100%|HC| |TAMS 0x000000070ce00000, 0x000000070ce00000| Complete -| 104|0x000000070d000000, 0x000000070d200000, 0x000000070d200000|100%|HS| |TAMS 0x000000070d000000, 0x000000070d000000| Complete -| 105|0x000000070d200000, 0x000000070d400000, 0x000000070d400000|100%|HS| |TAMS 0x000000070d200000, 0x000000070d200000| Complete -| 106|0x000000070d400000, 0x000000070d600000, 0x000000070d600000|100%|HC| |TAMS 0x000000070d400000, 0x000000070d400000| Complete -| 107|0x000000070d600000, 0x000000070d800000, 0x000000070d800000|100%|HC| |TAMS 0x000000070d600000, 0x000000070d600000| Complete -| 108|0x000000070d800000, 0x000000070da00000, 0x000000070da00000|100%|HS| |TAMS 0x000000070d800000, 0x000000070d800000| Complete -| 109|0x000000070da00000, 0x000000070dc00000, 0x000000070dc00000|100%|HS| |TAMS 0x000000070da00000, 0x000000070da00000| Complete -| 110|0x000000070dc00000, 0x000000070de00000, 0x000000070de00000|100%|HS| |TAMS 0x000000070dc00000, 0x000000070dc00000| Complete -| 111|0x000000070de00000, 0x000000070e000000, 0x000000070e000000|100%|HS| |TAMS 0x000000070de00000, 0x000000070de00000| Complete -| 112|0x000000070e000000, 0x000000070e200000, 0x000000070e200000|100%|HS| |TAMS 0x000000070e000000, 0x000000070e000000| Complete -| 113|0x000000070e200000, 0x000000070e400000, 0x000000070e400000|100%|HS| |TAMS 0x000000070e200000, 0x000000070e200000| Complete -| 114|0x000000070e400000, 0x000000070e600000, 0x000000070e600000|100%|HS| |TAMS 0x000000070e400000, 0x000000070e400000| Complete -| 115|0x000000070e600000, 0x000000070e800000, 0x000000070e800000|100%|HS| |TAMS 0x000000070e600000, 0x000000070e600000| Complete -| 116|0x000000070e800000, 0x000000070ea00000, 0x000000070ea00000|100%|HS| |TAMS 0x000000070e800000, 0x000000070e800000| Complete -| 117|0x000000070ea00000, 0x000000070ec00000, 0x000000070ec00000|100%|HS| |TAMS 0x000000070ea00000, 0x000000070ea00000| Complete -| 118|0x000000070ec00000, 0x000000070ee00000, 0x000000070ee00000|100%|HS| |TAMS 0x000000070ec00000, 0x000000070ec00000| Complete -| 119|0x000000070ee00000, 0x000000070f000000, 0x000000070f000000|100%|HS| |TAMS 0x000000070ee00000, 0x000000070ee00000| Complete -| 120|0x000000070f000000, 0x000000070f200000, 0x000000070f200000|100%|HS| |TAMS 0x000000070f000000, 0x000000070f000000| Complete -| 121|0x000000070f200000, 0x000000070f400000, 0x000000070f400000|100%|HS| |TAMS 0x000000070f200000, 0x000000070f200000| Complete -| 122|0x000000070f400000, 0x000000070f600000, 0x000000070f600000|100%|HS| |TAMS 0x000000070f400000, 0x000000070f400000| Complete -| 123|0x000000070f600000, 0x000000070f800000, 0x000000070f800000|100%|HS| |TAMS 0x000000070f600000, 0x000000070f600000| Complete -| 124|0x000000070f800000, 0x000000070fa00000, 0x000000070fa00000|100%|HS| |TAMS 0x000000070f800000, 0x000000070f800000| Complete -| 125|0x000000070fa00000, 0x000000070fc00000, 0x000000070fc00000|100%|HS| |TAMS 0x000000070fa00000, 0x000000070fa00000| Complete -| 126|0x000000070fc00000, 0x000000070fe00000, 0x000000070fe00000|100%|HS| |TAMS 0x000000070fc00000, 0x000000070fc00000| Complete -| 127|0x000000070fe00000, 0x0000000710000000, 0x0000000710000000|100%|HS| |TAMS 0x000000070fe00000, 0x000000070fe00000| Complete -| 128|0x0000000710000000, 0x0000000710200000, 0x0000000710200000|100%|HS| |TAMS 0x0000000710000000, 0x0000000710000000| Complete -| 129|0x0000000710200000, 0x0000000710400000, 0x0000000710400000|100%|HC| |TAMS 0x0000000710200000, 0x0000000710200000| Complete -| 130|0x0000000710400000, 0x0000000710600000, 0x0000000710600000|100%|HC| |TAMS 0x0000000710400000, 0x0000000710400000| Complete -| 131|0x0000000710600000, 0x0000000710800000, 0x0000000710800000|100%|HS| |TAMS 0x0000000710600000, 0x0000000710600000| Complete -| 132|0x0000000710800000, 0x0000000710a00000, 0x0000000710a00000|100%|HS| |TAMS 0x0000000710800000, 0x0000000710800000| Complete -| 133|0x0000000710a00000, 0x0000000710c00000, 0x0000000710c00000|100%|HC| |TAMS 0x0000000710a00000, 0x0000000710a00000| Complete -| 134|0x0000000710c00000, 0x0000000710e00000, 0x0000000710e00000|100%|HC| |TAMS 0x0000000710c00000, 0x0000000710c00000| Complete -| 135|0x0000000710e00000, 0x0000000711000000, 0x0000000711000000|100%|HS| |TAMS 0x0000000710e00000, 0x0000000710e00000| Complete -| 136|0x0000000711000000, 0x0000000711200000, 0x0000000711200000|100%|HS| |TAMS 0x0000000711000000, 0x0000000711000000| Complete -| 137|0x0000000711200000, 0x0000000711400000, 0x0000000711400000|100%|HS| |TAMS 0x0000000711200000, 0x0000000711200000| Complete -| 138|0x0000000711400000, 0x0000000711600000, 0x0000000711600000|100%|HS| |TAMS 0x0000000711400000, 0x0000000711400000| Complete -| 139|0x0000000711600000, 0x0000000711800000, 0x0000000711800000|100%|HS| |TAMS 0x0000000711600000, 0x0000000711600000| Complete -| 140|0x0000000711800000, 0x0000000711a00000, 0x0000000711a00000|100%|HS| |TAMS 0x0000000711800000, 0x0000000711800000| Complete -| 141|0x0000000711a00000, 0x0000000711c00000, 0x0000000711c00000|100%|HS| |TAMS 0x0000000711a00000, 0x0000000711a00000| Complete -| 142|0x0000000711c00000, 0x0000000711e00000, 0x0000000711e00000|100%|HS| |TAMS 0x0000000711c00000, 0x0000000711c00000| Complete -| 143|0x0000000711e00000, 0x0000000712000000, 0x0000000712000000|100%|HS| |TAMS 0x0000000711e00000, 0x0000000711e00000| Complete -| 144|0x0000000712000000, 0x0000000712200000, 0x0000000712200000|100%|HS| |TAMS 0x0000000712000000, 0x0000000712000000| Complete -| 145|0x0000000712200000, 0x0000000712200000, 0x0000000712400000| 0%| F| |TAMS 0x0000000712200000, 0x0000000712200000| Untracked -| 146|0x0000000712400000, 0x0000000712600000, 0x0000000712600000|100%| O| |TAMS 0x0000000712400000, 0x0000000712600000| Untracked -| 147|0x0000000712600000, 0x0000000712800000, 0x0000000712800000|100%| O| |TAMS 0x0000000712600000, 0x0000000712800000| Untracked -| 148|0x0000000712800000, 0x0000000712800000, 0x0000000712a00000| 0%| F| |TAMS 0x0000000712800000, 0x0000000712800000| Untracked -| 149|0x0000000712a00000, 0x0000000712a00000, 0x0000000712c00000| 0%| F| |TAMS 0x0000000712a00000, 0x0000000712a00000| Untracked -| 150|0x0000000712c00000, 0x0000000712dea400, 0x0000000712e00000| 95%| O| |TAMS 0x0000000712c00000, 0x0000000712d9d000| Untracked -| 151|0x0000000712e00000, 0x0000000713000000, 0x0000000713000000|100%| O| |TAMS 0x0000000712e00000, 0x0000000713000000| Untracked -| 152|0x0000000713000000, 0x0000000713200000, 0x0000000713200000|100%|HS| |TAMS 0x0000000713000000, 0x0000000713000000| Complete -| 153|0x0000000713200000, 0x0000000713400000, 0x0000000713400000|100%|HC| |TAMS 0x0000000713200000, 0x0000000713200000| Complete -| 154|0x0000000713400000, 0x0000000713600000, 0x0000000713600000|100%|HC| |TAMS 0x0000000713400000, 0x0000000713400000| Complete -| 155|0x0000000713600000, 0x0000000713600000, 0x0000000713800000| 0%| F| |TAMS 0x0000000713600000, 0x0000000713600000| Untracked -| 156|0x0000000713800000, 0x0000000713800000, 0x0000000713a00000| 0%| F| |TAMS 0x0000000713800000, 0x0000000713800000| Untracked -| 157|0x0000000713a00000, 0x0000000713a00000, 0x0000000713c00000| 0%| F| |TAMS 0x0000000713a00000, 0x0000000713a00000| Untracked -| 158|0x0000000713c00000, 0x0000000713c00000, 0x0000000713e00000| 0%| F| |TAMS 0x0000000713c00000, 0x0000000713c00000| Untracked -| 159|0x0000000713e00000, 0x0000000713e00000, 0x0000000714000000| 0%| F| |TAMS 0x0000000713e00000, 0x0000000713e00000| Untracked -| 160|0x0000000714000000, 0x0000000714000000, 0x0000000714200000| 0%| F| |TAMS 0x0000000714000000, 0x0000000714000000| Untracked -| 161|0x0000000714200000, 0x0000000714200000, 0x0000000714400000| 0%| F| |TAMS 0x0000000714200000, 0x0000000714200000| Untracked -| 162|0x0000000714400000, 0x0000000714400000, 0x0000000714600000| 0%| F| |TAMS 0x0000000714400000, 0x0000000714400000| Untracked -| 163|0x0000000714600000, 0x0000000714600000, 0x0000000714800000| 0%| F| |TAMS 0x0000000714600000, 0x0000000714600000| Untracked -| 164|0x0000000714800000, 0x0000000714800000, 0x0000000714a00000| 0%| F| |TAMS 0x0000000714800000, 0x0000000714800000| Untracked -| 165|0x0000000714a00000, 0x0000000714a00000, 0x0000000714c00000| 0%| F| |TAMS 0x0000000714a00000, 0x0000000714a00000| Untracked -| 166|0x0000000714c00000, 0x0000000714c00000, 0x0000000714e00000| 0%| F| |TAMS 0x0000000714c00000, 0x0000000714c00000| Untracked -| 167|0x0000000714e00000, 0x0000000714e00000, 0x0000000715000000| 0%| F| |TAMS 0x0000000714e00000, 0x0000000714e00000| Untracked -| 168|0x0000000715000000, 0x0000000715000000, 0x0000000715200000| 0%| F| |TAMS 0x0000000715000000, 0x0000000715000000| Untracked -| 169|0x0000000715200000, 0x0000000715200000, 0x0000000715400000| 0%| F| |TAMS 0x0000000715200000, 0x0000000715200000| Untracked -| 170|0x0000000715400000, 0x0000000715400000, 0x0000000715600000| 0%| F| |TAMS 0x0000000715400000, 0x0000000715400000| Untracked -| 171|0x0000000715600000, 0x0000000715600000, 0x0000000715800000| 0%| F| |TAMS 0x0000000715600000, 0x0000000715600000| Untracked -| 172|0x0000000715800000, 0x0000000715800000, 0x0000000715a00000| 0%| F| |TAMS 0x0000000715800000, 0x0000000715800000| Untracked -| 173|0x0000000715a00000, 0x0000000715a00000, 0x0000000715c00000| 0%| F| |TAMS 0x0000000715a00000, 0x0000000715a00000| Untracked -| 174|0x0000000715c00000, 0x0000000715c00000, 0x0000000715e00000| 0%| F| |TAMS 0x0000000715c00000, 0x0000000715c00000| Untracked -| 175|0x0000000715e00000, 0x0000000715e00000, 0x0000000716000000| 0%| F| |TAMS 0x0000000715e00000, 0x0000000715e00000| Untracked -| 176|0x0000000716000000, 0x0000000716000000, 0x0000000716200000| 0%| F| |TAMS 0x0000000716000000, 0x0000000716000000| Untracked -| 177|0x0000000716200000, 0x0000000716200000, 0x0000000716400000| 0%| F| |TAMS 0x0000000716200000, 0x0000000716200000| Untracked -| 178|0x0000000716400000, 0x0000000716400000, 0x0000000716600000| 0%| F| |TAMS 0x0000000716400000, 0x0000000716400000| Untracked -| 179|0x0000000716600000, 0x0000000716600000, 0x0000000716800000| 0%| F| |TAMS 0x0000000716600000, 0x0000000716600000| Untracked -| 180|0x0000000716800000, 0x0000000716800000, 0x0000000716a00000| 0%| F| |TAMS 0x0000000716800000, 0x0000000716800000| Untracked -| 181|0x0000000716a00000, 0x0000000716a00000, 0x0000000716c00000| 0%| F| |TAMS 0x0000000716a00000, 0x0000000716a00000| Untracked -| 182|0x0000000716c00000, 0x0000000716c00000, 0x0000000716e00000| 0%| F| |TAMS 0x0000000716c00000, 0x0000000716c00000| Untracked -| 183|0x0000000716e00000, 0x0000000716e00000, 0x0000000717000000| 0%| F| |TAMS 0x0000000716e00000, 0x0000000716e00000| Untracked -| 184|0x0000000717000000, 0x0000000717000000, 0x0000000717200000| 0%| F| |TAMS 0x0000000717000000, 0x0000000717000000| Untracked -| 185|0x0000000717200000, 0x0000000717200000, 0x0000000717400000| 0%| F| |TAMS 0x0000000717200000, 0x0000000717200000| Untracked -| 186|0x0000000717400000, 0x0000000717400000, 0x0000000717600000| 0%| F| |TAMS 0x0000000717400000, 0x0000000717400000| Untracked -| 187|0x0000000717600000, 0x0000000717600000, 0x0000000717800000| 0%| F| |TAMS 0x0000000717600000, 0x0000000717600000| Untracked -| 188|0x0000000717800000, 0x0000000717800000, 0x0000000717a00000| 0%| F| |TAMS 0x0000000717800000, 0x0000000717800000| Untracked -| 189|0x0000000717a00000, 0x0000000717a00000, 0x0000000717c00000| 0%| F| |TAMS 0x0000000717a00000, 0x0000000717a00000| Untracked -| 190|0x0000000717c00000, 0x0000000717c00000, 0x0000000717e00000| 0%| F| |TAMS 0x0000000717c00000, 0x0000000717c00000| Untracked -| 191|0x0000000717e00000, 0x0000000717e00000, 0x0000000718000000| 0%| F| |TAMS 0x0000000717e00000, 0x0000000717e00000| Untracked -| 192|0x0000000718000000, 0x0000000718000000, 0x0000000718200000| 0%| F| |TAMS 0x0000000718000000, 0x0000000718000000| Untracked -| 193|0x0000000718200000, 0x0000000718200000, 0x0000000718400000| 0%| F| |TAMS 0x0000000718200000, 0x0000000718200000| Untracked -| 194|0x0000000718400000, 0x0000000718400000, 0x0000000718600000| 0%| F| |TAMS 0x0000000718400000, 0x0000000718400000| Untracked -| 195|0x0000000718600000, 0x0000000718600000, 0x0000000718800000| 0%| F| |TAMS 0x0000000718600000, 0x0000000718600000| Untracked -| 196|0x0000000718800000, 0x0000000718800000, 0x0000000718a00000| 0%| F| |TAMS 0x0000000718800000, 0x0000000718800000| Untracked -| 197|0x0000000718a00000, 0x0000000718a00000, 0x0000000718c00000| 0%| F| |TAMS 0x0000000718a00000, 0x0000000718a00000| Untracked -| 198|0x0000000718c00000, 0x0000000718c00000, 0x0000000718e00000| 0%| F| |TAMS 0x0000000718c00000, 0x0000000718c00000| Untracked -| 199|0x0000000718e00000, 0x0000000718e00000, 0x0000000719000000| 0%| F| |TAMS 0x0000000718e00000, 0x0000000718e00000| Untracked -| 200|0x0000000719000000, 0x0000000719000000, 0x0000000719200000| 0%| F| |TAMS 0x0000000719000000, 0x0000000719000000| Untracked -| 201|0x0000000719200000, 0x0000000719200000, 0x0000000719400000| 0%| F| |TAMS 0x0000000719200000, 0x0000000719200000| Untracked -| 202|0x0000000719400000, 0x0000000719400000, 0x0000000719600000| 0%| F| |TAMS 0x0000000719400000, 0x0000000719400000| Untracked -| 203|0x0000000719600000, 0x0000000719600000, 0x0000000719800000| 0%| F| |TAMS 0x0000000719600000, 0x0000000719600000| Untracked -| 204|0x0000000719800000, 0x0000000719800000, 0x0000000719a00000| 0%| F| |TAMS 0x0000000719800000, 0x0000000719800000| Untracked -| 205|0x0000000719a00000, 0x0000000719a00000, 0x0000000719c00000| 0%| F| |TAMS 0x0000000719a00000, 0x0000000719a00000| Untracked -| 206|0x0000000719c00000, 0x0000000719c00000, 0x0000000719e00000| 0%| F| |TAMS 0x0000000719c00000, 0x0000000719c00000| Untracked -| 207|0x0000000719e00000, 0x0000000719e00000, 0x000000071a000000| 0%| F| |TAMS 0x0000000719e00000, 0x0000000719e00000| Untracked -| 208|0x000000071a000000, 0x000000071a000000, 0x000000071a200000| 0%| F| |TAMS 0x000000071a000000, 0x000000071a000000| Untracked -| 209|0x000000071a200000, 0x000000071a200000, 0x000000071a400000| 0%| F| |TAMS 0x000000071a200000, 0x000000071a200000| Untracked -| 210|0x000000071a400000, 0x000000071a400000, 0x000000071a600000| 0%| F| |TAMS 0x000000071a400000, 0x000000071a400000| Untracked -| 211|0x000000071a600000, 0x000000071a600000, 0x000000071a800000| 0%| F| |TAMS 0x000000071a600000, 0x000000071a600000| Untracked -| 212|0x000000071a800000, 0x000000071a800000, 0x000000071aa00000| 0%| F| |TAMS 0x000000071a800000, 0x000000071a800000| Untracked -| 213|0x000000071aa00000, 0x000000071aa00000, 0x000000071ac00000| 0%| F| |TAMS 0x000000071aa00000, 0x000000071aa00000| Untracked -| 214|0x000000071ac00000, 0x000000071ac00000, 0x000000071ae00000| 0%| F| |TAMS 0x000000071ac00000, 0x000000071ac00000| Untracked -| 215|0x000000071ae00000, 0x000000071ae00000, 0x000000071b000000| 0%| F| |TAMS 0x000000071ae00000, 0x000000071ae00000| Untracked -| 216|0x000000071b000000, 0x000000071b000000, 0x000000071b200000| 0%| F| |TAMS 0x000000071b000000, 0x000000071b000000| Untracked -| 217|0x000000071b200000, 0x000000071b200000, 0x000000071b400000| 0%| F| |TAMS 0x000000071b200000, 0x000000071b200000| Untracked -| 218|0x000000071b400000, 0x000000071b400000, 0x000000071b600000| 0%| F| |TAMS 0x000000071b400000, 0x000000071b400000| Untracked -| 219|0x000000071b600000, 0x000000071b600000, 0x000000071b800000| 0%| F| |TAMS 0x000000071b600000, 0x000000071b600000| Untracked -| 220|0x000000071b800000, 0x000000071b800000, 0x000000071ba00000| 0%| F| |TAMS 0x000000071b800000, 0x000000071b800000| Untracked -| 221|0x000000071ba00000, 0x000000071ba00000, 0x000000071bc00000| 0%| F| |TAMS 0x000000071ba00000, 0x000000071ba00000| Untracked -| 222|0x000000071bc00000, 0x000000071bc00000, 0x000000071be00000| 0%| F| |TAMS 0x000000071bc00000, 0x000000071bc00000| Untracked -| 223|0x000000071be00000, 0x000000071be00000, 0x000000071c000000| 0%| F| |TAMS 0x000000071be00000, 0x000000071be00000| Untracked -| 224|0x000000071c000000, 0x000000071c000000, 0x000000071c200000| 0%| F| |TAMS 0x000000071c000000, 0x000000071c000000| Untracked -| 225|0x000000071c200000, 0x000000071c200000, 0x000000071c400000| 0%| F| |TAMS 0x000000071c200000, 0x000000071c200000| Untracked -| 226|0x000000071c400000, 0x000000071c400000, 0x000000071c600000| 0%| F| |TAMS 0x000000071c400000, 0x000000071c400000| Untracked -| 227|0x000000071c600000, 0x000000071c600000, 0x000000071c800000| 0%| F| |TAMS 0x000000071c600000, 0x000000071c600000| Untracked -| 228|0x000000071c800000, 0x000000071c800000, 0x000000071ca00000| 0%| F| |TAMS 0x000000071c800000, 0x000000071c800000| Untracked -| 229|0x000000071ca00000, 0x000000071ca00000, 0x000000071cc00000| 0%| F| |TAMS 0x000000071ca00000, 0x000000071ca00000| Untracked -| 230|0x000000071cc00000, 0x000000071cc00000, 0x000000071ce00000| 0%| F| |TAMS 0x000000071cc00000, 0x000000071cc00000| Untracked -| 231|0x000000071ce00000, 0x000000071ce00000, 0x000000071d000000| 0%| F| |TAMS 0x000000071ce00000, 0x000000071ce00000| Untracked -| 232|0x000000071d000000, 0x000000071d000000, 0x000000071d200000| 0%| F| |TAMS 0x000000071d000000, 0x000000071d000000| Untracked -| 233|0x000000071d200000, 0x000000071d200000, 0x000000071d400000| 0%| F| |TAMS 0x000000071d200000, 0x000000071d200000| Untracked -| 234|0x000000071d400000, 0x000000071d400000, 0x000000071d600000| 0%| F| |TAMS 0x000000071d400000, 0x000000071d400000| Untracked -| 235|0x000000071d600000, 0x000000071d600000, 0x000000071d800000| 0%| F| |TAMS 0x000000071d600000, 0x000000071d600000| Untracked -| 236|0x000000071d800000, 0x000000071d800000, 0x000000071da00000| 0%| F| |TAMS 0x000000071d800000, 0x000000071d800000| Untracked -| 237|0x000000071da00000, 0x000000071da00000, 0x000000071dc00000| 0%| F| |TAMS 0x000000071da00000, 0x000000071da00000| Untracked -| 238|0x000000071dc00000, 0x000000071dc00000, 0x000000071de00000| 0%| F| |TAMS 0x000000071dc00000, 0x000000071dc00000| Untracked -| 239|0x000000071de00000, 0x000000071de00000, 0x000000071e000000| 0%| F| |TAMS 0x000000071de00000, 0x000000071de00000| Untracked -| 240|0x000000071e000000, 0x000000071e000000, 0x000000071e200000| 0%| F| |TAMS 0x000000071e000000, 0x000000071e000000| Untracked -| 241|0x000000071e200000, 0x000000071e200000, 0x000000071e400000| 0%| F| |TAMS 0x000000071e200000, 0x000000071e200000| Untracked -| 242|0x000000071e400000, 0x000000071e400000, 0x000000071e600000| 0%| F| |TAMS 0x000000071e400000, 0x000000071e400000| Untracked -| 243|0x000000071e600000, 0x000000071e600000, 0x000000071e800000| 0%| F| |TAMS 0x000000071e600000, 0x000000071e600000| Untracked -| 244|0x000000071e800000, 0x000000071e800000, 0x000000071ea00000| 0%| F| |TAMS 0x000000071e800000, 0x000000071e800000| Untracked -| 245|0x000000071ea00000, 0x000000071ea00000, 0x000000071ec00000| 0%| F| |TAMS 0x000000071ea00000, 0x000000071ea00000| Untracked -| 246|0x000000071ec00000, 0x000000071ec00000, 0x000000071ee00000| 0%| F| |TAMS 0x000000071ec00000, 0x000000071ec00000| Untracked -| 247|0x000000071ee00000, 0x000000071ee00000, 0x000000071f000000| 0%| F| |TAMS 0x000000071ee00000, 0x000000071ee00000| Untracked -| 248|0x000000071f000000, 0x000000071f000000, 0x000000071f200000| 0%| F| |TAMS 0x000000071f000000, 0x000000071f000000| Untracked -| 249|0x000000071f200000, 0x000000071f200000, 0x000000071f400000| 0%| F| |TAMS 0x000000071f200000, 0x000000071f200000| Untracked -| 250|0x000000071f400000, 0x000000071f400000, 0x000000071f600000| 0%| F| |TAMS 0x000000071f400000, 0x000000071f400000| Untracked -| 251|0x000000071f600000, 0x000000071f600000, 0x000000071f800000| 0%| F| |TAMS 0x000000071f600000, 0x000000071f600000| Untracked -| 252|0x000000071f800000, 0x000000071f800000, 0x000000071fa00000| 0%| F| |TAMS 0x000000071f800000, 0x000000071f800000| Untracked -| 253|0x000000071fa00000, 0x000000071fa00000, 0x000000071fc00000| 0%| F| |TAMS 0x000000071fa00000, 0x000000071fa00000| Untracked -| 254|0x000000071fc00000, 0x000000071fc00000, 0x000000071fe00000| 0%| F| |TAMS 0x000000071fc00000, 0x000000071fc00000| Untracked -| 255|0x000000071fe00000, 0x000000071fe00000, 0x0000000720000000| 0%| F| |TAMS 0x000000071fe00000, 0x000000071fe00000| Untracked -| 256|0x0000000720000000, 0x0000000720000000, 0x0000000720200000| 0%| F| |TAMS 0x0000000720000000, 0x0000000720000000| Untracked -| 257|0x0000000720200000, 0x0000000720200000, 0x0000000720400000| 0%| F| |TAMS 0x0000000720200000, 0x0000000720200000| Untracked -| 258|0x0000000720400000, 0x0000000720400000, 0x0000000720600000| 0%| F| |TAMS 0x0000000720400000, 0x0000000720400000| Untracked -| 259|0x0000000720600000, 0x0000000720600000, 0x0000000720800000| 0%| F| |TAMS 0x0000000720600000, 0x0000000720600000| Untracked -| 260|0x0000000720800000, 0x0000000720800000, 0x0000000720a00000| 0%| F| |TAMS 0x0000000720800000, 0x0000000720800000| Untracked -| 261|0x0000000720a00000, 0x0000000720a00000, 0x0000000720c00000| 0%| F| |TAMS 0x0000000720a00000, 0x0000000720a00000| Untracked -| 262|0x0000000720c00000, 0x0000000720c00000, 0x0000000720e00000| 0%| F| |TAMS 0x0000000720c00000, 0x0000000720c00000| Untracked -| 263|0x0000000720e00000, 0x0000000720e00000, 0x0000000721000000| 0%| F| |TAMS 0x0000000720e00000, 0x0000000720e00000| Untracked -| 264|0x0000000721000000, 0x0000000721000000, 0x0000000721200000| 0%| F| |TAMS 0x0000000721000000, 0x0000000721000000| Untracked -| 265|0x0000000721200000, 0x0000000721200000, 0x0000000721400000| 0%| F| |TAMS 0x0000000721200000, 0x0000000721200000| Untracked -| 266|0x0000000721400000, 0x0000000721400000, 0x0000000721600000| 0%| F| |TAMS 0x0000000721400000, 0x0000000721400000| Untracked -| 267|0x0000000721600000, 0x0000000721600000, 0x0000000721800000| 0%| F| |TAMS 0x0000000721600000, 0x0000000721600000| Untracked -| 268|0x0000000721800000, 0x0000000721800000, 0x0000000721a00000| 0%| F| |TAMS 0x0000000721800000, 0x0000000721800000| Untracked -| 269|0x0000000721a00000, 0x0000000721a00000, 0x0000000721c00000| 0%| F| |TAMS 0x0000000721a00000, 0x0000000721a00000| Untracked -| 270|0x0000000721c00000, 0x0000000721c00000, 0x0000000721e00000| 0%| F| |TAMS 0x0000000721c00000, 0x0000000721c00000| Untracked -| 271|0x0000000721e00000, 0x0000000721e00000, 0x0000000722000000| 0%| F| |TAMS 0x0000000721e00000, 0x0000000721e00000| Untracked -| 272|0x0000000722000000, 0x0000000722000000, 0x0000000722200000| 0%| F| |TAMS 0x0000000722000000, 0x0000000722000000| Untracked -| 273|0x0000000722200000, 0x0000000722200000, 0x0000000722400000| 0%| F| |TAMS 0x0000000722200000, 0x0000000722200000| Untracked -| 274|0x0000000722400000, 0x0000000722400000, 0x0000000722600000| 0%| F| |TAMS 0x0000000722400000, 0x0000000722400000| Untracked -| 275|0x0000000722600000, 0x0000000722600000, 0x0000000722800000| 0%| F| |TAMS 0x0000000722600000, 0x0000000722600000| Untracked -| 276|0x0000000722800000, 0x0000000722800000, 0x0000000722a00000| 0%| F| |TAMS 0x0000000722800000, 0x0000000722800000| Untracked -| 277|0x0000000722a00000, 0x0000000722a00000, 0x0000000722c00000| 0%| F| |TAMS 0x0000000722a00000, 0x0000000722a00000| Untracked -| 278|0x0000000722c00000, 0x0000000722c00000, 0x0000000722e00000| 0%| F| |TAMS 0x0000000722c00000, 0x0000000722c00000| Untracked -| 279|0x0000000722e00000, 0x0000000722e00000, 0x0000000723000000| 0%| F| |TAMS 0x0000000722e00000, 0x0000000722e00000| Untracked -| 280|0x0000000723000000, 0x0000000723000000, 0x0000000723200000| 0%| F| |TAMS 0x0000000723000000, 0x0000000723000000| Untracked -| 281|0x0000000723200000, 0x0000000723200000, 0x0000000723400000| 0%| F| |TAMS 0x0000000723200000, 0x0000000723200000| Untracked -| 282|0x0000000723400000, 0x0000000723400000, 0x0000000723600000| 0%| F| |TAMS 0x0000000723400000, 0x0000000723400000| Untracked -| 283|0x0000000723600000, 0x0000000723600000, 0x0000000723800000| 0%| F| |TAMS 0x0000000723600000, 0x0000000723600000| Untracked -| 284|0x0000000723800000, 0x0000000723800000, 0x0000000723a00000| 0%| F| |TAMS 0x0000000723800000, 0x0000000723800000| Untracked -| 285|0x0000000723a00000, 0x0000000723a00000, 0x0000000723c00000| 0%| F| |TAMS 0x0000000723a00000, 0x0000000723a00000| Untracked -| 286|0x0000000723c00000, 0x0000000723c00000, 0x0000000723e00000| 0%| F| |TAMS 0x0000000723c00000, 0x0000000723c00000| Untracked -| 287|0x0000000723e00000, 0x0000000723e00000, 0x0000000724000000| 0%| F| |TAMS 0x0000000723e00000, 0x0000000723e00000| Untracked -| 288|0x0000000724000000, 0x0000000724000000, 0x0000000724200000| 0%| F| |TAMS 0x0000000724000000, 0x0000000724000000| Untracked -| 289|0x0000000724200000, 0x0000000724200000, 0x0000000724400000| 0%| F| |TAMS 0x0000000724200000, 0x0000000724200000| Untracked -| 290|0x0000000724400000, 0x0000000724400000, 0x0000000724600000| 0%| F| |TAMS 0x0000000724400000, 0x0000000724400000| Untracked -| 291|0x0000000724600000, 0x0000000724600000, 0x0000000724800000| 0%| F| |TAMS 0x0000000724600000, 0x0000000724600000| Untracked -| 292|0x0000000724800000, 0x0000000724800000, 0x0000000724a00000| 0%| F| |TAMS 0x0000000724800000, 0x0000000724800000| Untracked -| 293|0x0000000724a00000, 0x0000000724a00000, 0x0000000724c00000| 0%| F| |TAMS 0x0000000724a00000, 0x0000000724a00000| Untracked -| 294|0x0000000724c00000, 0x0000000724c00000, 0x0000000724e00000| 0%| F| |TAMS 0x0000000724c00000, 0x0000000724c00000| Untracked -| 295|0x0000000724e00000, 0x0000000724e00000, 0x0000000725000000| 0%| F| |TAMS 0x0000000724e00000, 0x0000000724e00000| Untracked -| 296|0x0000000725000000, 0x0000000725000000, 0x0000000725200000| 0%| F| |TAMS 0x0000000725000000, 0x0000000725000000| Untracked -| 297|0x0000000725200000, 0x0000000725200000, 0x0000000725400000| 0%| F| |TAMS 0x0000000725200000, 0x0000000725200000| Untracked -| 298|0x0000000725400000, 0x0000000725400000, 0x0000000725600000| 0%| F| |TAMS 0x0000000725400000, 0x0000000725400000| Untracked -| 299|0x0000000725600000, 0x0000000725600000, 0x0000000725800000| 0%| F| |TAMS 0x0000000725600000, 0x0000000725600000| Untracked -| 300|0x0000000725800000, 0x0000000725800000, 0x0000000725a00000| 0%| F| |TAMS 0x0000000725800000, 0x0000000725800000| Untracked -| 301|0x0000000725a00000, 0x0000000725a00000, 0x0000000725c00000| 0%| F| |TAMS 0x0000000725a00000, 0x0000000725a00000| Untracked -| 302|0x0000000725c00000, 0x0000000725c00000, 0x0000000725e00000| 0%| F| |TAMS 0x0000000725c00000, 0x0000000725c00000| Untracked -| 303|0x0000000725e00000, 0x0000000725e00000, 0x0000000726000000| 0%| F| |TAMS 0x0000000725e00000, 0x0000000725e00000| Untracked -| 304|0x0000000726000000, 0x0000000726000000, 0x0000000726200000| 0%| F| |TAMS 0x0000000726000000, 0x0000000726000000| Untracked -| 305|0x0000000726200000, 0x0000000726200000, 0x0000000726400000| 0%| F| |TAMS 0x0000000726200000, 0x0000000726200000| Untracked -| 306|0x0000000726400000, 0x0000000726400000, 0x0000000726600000| 0%| F| |TAMS 0x0000000726400000, 0x0000000726400000| Untracked -| 307|0x0000000726600000, 0x0000000726600000, 0x0000000726800000| 0%| F| |TAMS 0x0000000726600000, 0x0000000726600000| Untracked -| 308|0x0000000726800000, 0x0000000726800000, 0x0000000726a00000| 0%| F| |TAMS 0x0000000726800000, 0x0000000726800000| Untracked -| 309|0x0000000726a00000, 0x0000000726a00000, 0x0000000726c00000| 0%| F| |TAMS 0x0000000726a00000, 0x0000000726a00000| Untracked -| 310|0x0000000726c00000, 0x0000000726c00000, 0x0000000726e00000| 0%| F| |TAMS 0x0000000726c00000, 0x0000000726c00000| Untracked -| 311|0x0000000726e00000, 0x0000000726e00000, 0x0000000727000000| 0%| F| |TAMS 0x0000000726e00000, 0x0000000726e00000| Untracked -| 312|0x0000000727000000, 0x0000000727000000, 0x0000000727200000| 0%| F| |TAMS 0x0000000727000000, 0x0000000727000000| Untracked -| 313|0x0000000727200000, 0x0000000727200000, 0x0000000727400000| 0%| F| |TAMS 0x0000000727200000, 0x0000000727200000| Untracked -| 314|0x0000000727400000, 0x0000000727400000, 0x0000000727600000| 0%| F| |TAMS 0x0000000727400000, 0x0000000727400000| Untracked -| 315|0x0000000727600000, 0x0000000727600000, 0x0000000727800000| 0%| F| |TAMS 0x0000000727600000, 0x0000000727600000| Untracked -| 316|0x0000000727800000, 0x0000000727800000, 0x0000000727a00000| 0%| F| |TAMS 0x0000000727800000, 0x0000000727800000| Untracked -| 317|0x0000000727a00000, 0x0000000727a00000, 0x0000000727c00000| 0%| F| |TAMS 0x0000000727a00000, 0x0000000727a00000| Untracked -| 318|0x0000000727c00000, 0x0000000727c00000, 0x0000000727e00000| 0%| F| |TAMS 0x0000000727c00000, 0x0000000727c00000| Untracked -| 319|0x0000000727e00000, 0x0000000727e00000, 0x0000000728000000| 0%| F| |TAMS 0x0000000727e00000, 0x0000000727e00000| Untracked -| 320|0x0000000728000000, 0x0000000728000000, 0x0000000728200000| 0%| F| |TAMS 0x0000000728000000, 0x0000000728000000| Untracked -| 321|0x0000000728200000, 0x0000000728200000, 0x0000000728400000| 0%| F| |TAMS 0x0000000728200000, 0x0000000728200000| Untracked -| 322|0x0000000728400000, 0x0000000728400000, 0x0000000728600000| 0%| F| |TAMS 0x0000000728400000, 0x0000000728400000| Untracked -| 323|0x0000000728600000, 0x0000000728600000, 0x0000000728800000| 0%| F| |TAMS 0x0000000728600000, 0x0000000728600000| Untracked -| 324|0x0000000728800000, 0x0000000728800000, 0x0000000728a00000| 0%| F| |TAMS 0x0000000728800000, 0x0000000728800000| Untracked -| 325|0x0000000728a00000, 0x0000000728a00000, 0x0000000728c00000| 0%| F| |TAMS 0x0000000728a00000, 0x0000000728a00000| Untracked -| 326|0x0000000728c00000, 0x0000000728c00000, 0x0000000728e00000| 0%| F| |TAMS 0x0000000728c00000, 0x0000000728c00000| Untracked -| 327|0x0000000728e00000, 0x0000000728e00000, 0x0000000729000000| 0%| F| |TAMS 0x0000000728e00000, 0x0000000728e00000| Untracked -| 328|0x0000000729000000, 0x0000000729000000, 0x0000000729200000| 0%| F| |TAMS 0x0000000729000000, 0x0000000729000000| Untracked -| 329|0x0000000729200000, 0x0000000729200000, 0x0000000729400000| 0%| F| |TAMS 0x0000000729200000, 0x0000000729200000| Untracked -| 330|0x0000000729400000, 0x0000000729400000, 0x0000000729600000| 0%| F| |TAMS 0x0000000729400000, 0x0000000729400000| Untracked -| 331|0x0000000729600000, 0x0000000729600000, 0x0000000729800000| 0%| F| |TAMS 0x0000000729600000, 0x0000000729600000| Untracked -| 332|0x0000000729800000, 0x0000000729800000, 0x0000000729a00000| 0%| F| |TAMS 0x0000000729800000, 0x0000000729800000| Untracked -| 333|0x0000000729a00000, 0x0000000729a00000, 0x0000000729c00000| 0%| F| |TAMS 0x0000000729a00000, 0x0000000729a00000| Untracked -| 334|0x0000000729c00000, 0x0000000729c00000, 0x0000000729e00000| 0%| F| |TAMS 0x0000000729c00000, 0x0000000729c00000| Untracked -| 335|0x0000000729e00000, 0x0000000729e00000, 0x000000072a000000| 0%| F| |TAMS 0x0000000729e00000, 0x0000000729e00000| Untracked -| 336|0x000000072a000000, 0x000000072a000000, 0x000000072a200000| 0%| F| |TAMS 0x000000072a000000, 0x000000072a000000| Untracked -| 337|0x000000072a200000, 0x000000072a200000, 0x000000072a400000| 0%| F| |TAMS 0x000000072a200000, 0x000000072a200000| Untracked -| 338|0x000000072a400000, 0x000000072a400000, 0x000000072a600000| 0%| F| |TAMS 0x000000072a400000, 0x000000072a400000| Untracked -| 339|0x000000072a600000, 0x000000072a600000, 0x000000072a800000| 0%| F| |TAMS 0x000000072a600000, 0x000000072a600000| Untracked -| 340|0x000000072a800000, 0x000000072a800000, 0x000000072aa00000| 0%| F| |TAMS 0x000000072a800000, 0x000000072a800000| Untracked -| 341|0x000000072aa00000, 0x000000072aa00000, 0x000000072ac00000| 0%| F| |TAMS 0x000000072aa00000, 0x000000072aa00000| Untracked -| 342|0x000000072ac00000, 0x000000072ac00000, 0x000000072ae00000| 0%| F| |TAMS 0x000000072ac00000, 0x000000072ac00000| Untracked -| 343|0x000000072ae00000, 0x000000072ae00000, 0x000000072b000000| 0%| F| |TAMS 0x000000072ae00000, 0x000000072ae00000| Untracked -| 344|0x000000072b000000, 0x000000072b000000, 0x000000072b200000| 0%| F| |TAMS 0x000000072b000000, 0x000000072b000000| Untracked -| 345|0x000000072b200000, 0x000000072b200000, 0x000000072b400000| 0%| F| |TAMS 0x000000072b200000, 0x000000072b200000| Untracked -| 346|0x000000072b400000, 0x000000072b400000, 0x000000072b600000| 0%| F| |TAMS 0x000000072b400000, 0x000000072b400000| Untracked -| 347|0x000000072b600000, 0x000000072b600000, 0x000000072b800000| 0%| F| |TAMS 0x000000072b600000, 0x000000072b600000| Untracked -| 348|0x000000072b800000, 0x000000072b800000, 0x000000072ba00000| 0%| F| |TAMS 0x000000072b800000, 0x000000072b800000| Untracked -| 349|0x000000072ba00000, 0x000000072ba00000, 0x000000072bc00000| 0%| F| |TAMS 0x000000072ba00000, 0x000000072ba00000| Untracked -| 350|0x000000072bc00000, 0x000000072bc00000, 0x000000072be00000| 0%| F| |TAMS 0x000000072bc00000, 0x000000072bc00000| Untracked -| 351|0x000000072be00000, 0x000000072be00000, 0x000000072c000000| 0%| F| |TAMS 0x000000072be00000, 0x000000072be00000| Untracked -| 352|0x000000072c000000, 0x000000072c000000, 0x000000072c200000| 0%| F| |TAMS 0x000000072c000000, 0x000000072c000000| Untracked -| 353|0x000000072c200000, 0x000000072c200000, 0x000000072c400000| 0%| F| |TAMS 0x000000072c200000, 0x000000072c200000| Untracked -| 354|0x000000072c400000, 0x000000072c400000, 0x000000072c600000| 0%| F| |TAMS 0x000000072c400000, 0x000000072c400000| Untracked -| 355|0x000000072c600000, 0x000000072c600000, 0x000000072c800000| 0%| F| |TAMS 0x000000072c600000, 0x000000072c600000| Untracked -| 356|0x000000072c800000, 0x000000072c800000, 0x000000072ca00000| 0%| F| |TAMS 0x000000072c800000, 0x000000072c800000| Untracked -| 357|0x000000072ca00000, 0x000000072ca00000, 0x000000072cc00000| 0%| F| |TAMS 0x000000072ca00000, 0x000000072ca00000| Untracked -| 358|0x000000072cc00000, 0x000000072cc00000, 0x000000072ce00000| 0%| F| |TAMS 0x000000072cc00000, 0x000000072cc00000| Untracked -| 359|0x000000072ce00000, 0x000000072ce00000, 0x000000072d000000| 0%| F| |TAMS 0x000000072ce00000, 0x000000072ce00000| Untracked -| 360|0x000000072d000000, 0x000000072d000000, 0x000000072d200000| 0%| F| |TAMS 0x000000072d000000, 0x000000072d000000| Untracked -| 361|0x000000072d200000, 0x000000072d200000, 0x000000072d400000| 0%| F| |TAMS 0x000000072d200000, 0x000000072d200000| Untracked -| 362|0x000000072d400000, 0x000000072d400000, 0x000000072d600000| 0%| F| |TAMS 0x000000072d400000, 0x000000072d400000| Untracked -| 363|0x000000072d600000, 0x000000072d600000, 0x000000072d800000| 0%| F| |TAMS 0x000000072d600000, 0x000000072d600000| Untracked -| 364|0x000000072d800000, 0x000000072d800000, 0x000000072da00000| 0%| F| |TAMS 0x000000072d800000, 0x000000072d800000| Untracked -| 365|0x000000072da00000, 0x000000072da00000, 0x000000072dc00000| 0%| F| |TAMS 0x000000072da00000, 0x000000072da00000| Untracked -| 366|0x000000072dc00000, 0x000000072dc00000, 0x000000072de00000| 0%| F| |TAMS 0x000000072dc00000, 0x000000072dc00000| Untracked -| 367|0x000000072de00000, 0x000000072de00000, 0x000000072e000000| 0%| F| |TAMS 0x000000072de00000, 0x000000072de00000| Untracked -| 368|0x000000072e000000, 0x000000072e000000, 0x000000072e200000| 0%| F| |TAMS 0x000000072e000000, 0x000000072e000000| Untracked -| 369|0x000000072e200000, 0x000000072e200000, 0x000000072e400000| 0%| F| |TAMS 0x000000072e200000, 0x000000072e200000| Untracked -| 370|0x000000072e400000, 0x000000072e400000, 0x000000072e600000| 0%| F| |TAMS 0x000000072e400000, 0x000000072e400000| Untracked -| 371|0x000000072e600000, 0x000000072e600000, 0x000000072e800000| 0%| F| |TAMS 0x000000072e600000, 0x000000072e600000| Untracked -| 372|0x000000072e800000, 0x000000072e800000, 0x000000072ea00000| 0%| F| |TAMS 0x000000072e800000, 0x000000072e800000| Untracked -| 373|0x000000072ea00000, 0x000000072ea00000, 0x000000072ec00000| 0%| F| |TAMS 0x000000072ea00000, 0x000000072ea00000| Untracked -| 374|0x000000072ec00000, 0x000000072ec00000, 0x000000072ee00000| 0%| F| |TAMS 0x000000072ec00000, 0x000000072ec00000| Untracked -| 375|0x000000072ee00000, 0x000000072ee00000, 0x000000072f000000| 0%| F| |TAMS 0x000000072ee00000, 0x000000072ee00000| Untracked -| 376|0x000000072f000000, 0x000000072f000000, 0x000000072f200000| 0%| F| |TAMS 0x000000072f000000, 0x000000072f000000| Untracked -| 377|0x000000072f200000, 0x000000072f200000, 0x000000072f400000| 0%| F| |TAMS 0x000000072f200000, 0x000000072f200000| Untracked -| 378|0x000000072f400000, 0x000000072f400000, 0x000000072f600000| 0%| F| |TAMS 0x000000072f400000, 0x000000072f400000| Untracked -| 379|0x000000072f600000, 0x000000072f600000, 0x000000072f800000| 0%| F| |TAMS 0x000000072f600000, 0x000000072f600000| Untracked -| 380|0x000000072f800000, 0x000000072f800000, 0x000000072fa00000| 0%| F| |TAMS 0x000000072f800000, 0x000000072f800000| Untracked -| 381|0x000000072fa00000, 0x000000072fa00000, 0x000000072fc00000| 0%| F| |TAMS 0x000000072fa00000, 0x000000072fa00000| Untracked -| 382|0x000000072fc00000, 0x000000072fc00000, 0x000000072fe00000| 0%| F| |TAMS 0x000000072fc00000, 0x000000072fc00000| Untracked -| 383|0x000000072fe00000, 0x000000072fe00000, 0x0000000730000000| 0%| F| |TAMS 0x000000072fe00000, 0x000000072fe00000| Untracked -| 384|0x0000000730000000, 0x0000000730000000, 0x0000000730200000| 0%| F| |TAMS 0x0000000730000000, 0x0000000730000000| Untracked -| 385|0x0000000730200000, 0x0000000730200000, 0x0000000730400000| 0%| F| |TAMS 0x0000000730200000, 0x0000000730200000| Untracked -| 386|0x0000000730400000, 0x0000000730400000, 0x0000000730600000| 0%| F| |TAMS 0x0000000730400000, 0x0000000730400000| Untracked -| 387|0x0000000730600000, 0x0000000730600000, 0x0000000730800000| 0%| F| |TAMS 0x0000000730600000, 0x0000000730600000| Untracked -| 388|0x0000000730800000, 0x0000000730800000, 0x0000000730a00000| 0%| F| |TAMS 0x0000000730800000, 0x0000000730800000| Untracked -| 389|0x0000000730a00000, 0x0000000730a00000, 0x0000000730c00000| 0%| F| |TAMS 0x0000000730a00000, 0x0000000730a00000| Untracked -| 390|0x0000000730c00000, 0x0000000730c00000, 0x0000000730e00000| 0%| F| |TAMS 0x0000000730c00000, 0x0000000730c00000| Untracked -| 391|0x0000000730e00000, 0x0000000730e00000, 0x0000000731000000| 0%| F| |TAMS 0x0000000730e00000, 0x0000000730e00000| Untracked -| 392|0x0000000731000000, 0x0000000731000000, 0x0000000731200000| 0%| F| |TAMS 0x0000000731000000, 0x0000000731000000| Untracked -| 393|0x0000000731200000, 0x0000000731200000, 0x0000000731400000| 0%| F| |TAMS 0x0000000731200000, 0x0000000731200000| Untracked -| 394|0x0000000731400000, 0x0000000731400000, 0x0000000731600000| 0%| F| |TAMS 0x0000000731400000, 0x0000000731400000| Untracked -| 395|0x0000000731600000, 0x0000000731600000, 0x0000000731800000| 0%| F| |TAMS 0x0000000731600000, 0x0000000731600000| Untracked -| 396|0x0000000731800000, 0x0000000731800000, 0x0000000731a00000| 0%| F| |TAMS 0x0000000731800000, 0x0000000731800000| Untracked -| 397|0x0000000731a00000, 0x0000000731a00000, 0x0000000731c00000| 0%| F| |TAMS 0x0000000731a00000, 0x0000000731a00000| Untracked -| 398|0x0000000731c00000, 0x0000000731c9bf20, 0x0000000731e00000| 30%| S|CS|TAMS 0x0000000731c00000, 0x0000000731c00000| Complete -| 399|0x0000000731e00000, 0x0000000732000000, 0x0000000732000000|100%| S|CS|TAMS 0x0000000731e00000, 0x0000000731e00000| Complete -| 400|0x0000000732000000, 0x0000000732200000, 0x0000000732200000|100%| S|CS|TAMS 0x0000000732000000, 0x0000000732000000| Complete -| 401|0x0000000732200000, 0x0000000732400000, 0x0000000732400000|100%| S|CS|TAMS 0x0000000732200000, 0x0000000732200000| Complete -| 402|0x0000000732400000, 0x0000000732400000, 0x0000000732600000| 0%| F| |TAMS 0x0000000732400000, 0x0000000732400000| Untracked -| 403|0x0000000732600000, 0x0000000732600000, 0x0000000732800000| 0%| F| |TAMS 0x0000000732600000, 0x0000000732600000| Untracked -| 404|0x0000000732800000, 0x0000000732800000, 0x0000000732a00000| 0%| F| |TAMS 0x0000000732800000, 0x0000000732800000| Untracked -| 405|0x0000000732a00000, 0x0000000732a00000, 0x0000000732c00000| 0%| F| |TAMS 0x0000000732a00000, 0x0000000732a00000| Untracked -| 406|0x0000000732c00000, 0x0000000732c00000, 0x0000000732e00000| 0%| F| |TAMS 0x0000000732c00000, 0x0000000732c00000| Untracked -| 407|0x0000000732e00000, 0x0000000732e00000, 0x0000000733000000| 0%| F| |TAMS 0x0000000732e00000, 0x0000000732e00000| Untracked -| 408|0x0000000733000000, 0x000000073308e4c0, 0x0000000733200000| 27%| E| |TAMS 0x0000000733000000, 0x0000000733000000| Complete -| 409|0x0000000733200000, 0x0000000733400000, 0x0000000733400000|100%| E|CS|TAMS 0x0000000733200000, 0x0000000733200000| Complete -| 410|0x0000000733400000, 0x0000000733600000, 0x0000000733600000|100%| E|CS|TAMS 0x0000000733400000, 0x0000000733400000| Complete -| 411|0x0000000733600000, 0x0000000733800000, 0x0000000733800000|100%| E|CS|TAMS 0x0000000733600000, 0x0000000733600000| Complete -| 412|0x0000000733800000, 0x0000000733a00000, 0x0000000733a00000|100%| E|CS|TAMS 0x0000000733800000, 0x0000000733800000| Complete -| 413|0x0000000733a00000, 0x0000000733c00000, 0x0000000733c00000|100%| E|CS|TAMS 0x0000000733a00000, 0x0000000733a00000| Complete -| 414|0x0000000733c00000, 0x0000000733e00000, 0x0000000733e00000|100%| E|CS|TAMS 0x0000000733c00000, 0x0000000733c00000| Complete -| 415|0x0000000733e00000, 0x0000000734000000, 0x0000000734000000|100%| E| |TAMS 0x0000000733e00000, 0x0000000733e00000| Complete -| 416|0x0000000734000000, 0x0000000734200000, 0x0000000734200000|100%| E|CS|TAMS 0x0000000734000000, 0x0000000734000000| Complete -| 417|0x0000000734200000, 0x0000000734400000, 0x0000000734400000|100%| E|CS|TAMS 0x0000000734200000, 0x0000000734200000| Complete -| 418|0x0000000734400000, 0x0000000734600000, 0x0000000734600000|100%| E|CS|TAMS 0x0000000734400000, 0x0000000734400000| Complete -| 419|0x0000000734600000, 0x0000000734800000, 0x0000000734800000|100%| E|CS|TAMS 0x0000000734600000, 0x0000000734600000| Complete -| 420|0x0000000734800000, 0x0000000734a00000, 0x0000000734a00000|100%| E|CS|TAMS 0x0000000734800000, 0x0000000734800000| Complete -| 421|0x0000000734a00000, 0x0000000734c00000, 0x0000000734c00000|100%| E|CS|TAMS 0x0000000734a00000, 0x0000000734a00000| Complete -| 422|0x0000000734c00000, 0x0000000734e00000, 0x0000000734e00000|100%| E|CS|TAMS 0x0000000734c00000, 0x0000000734c00000| Complete -| 423|0x0000000734e00000, 0x0000000735000000, 0x0000000735000000|100%| E|CS|TAMS 0x0000000734e00000, 0x0000000734e00000| Complete -| 424|0x0000000735000000, 0x0000000735200000, 0x0000000735200000|100%| E|CS|TAMS 0x0000000735000000, 0x0000000735000000| Complete -| 425|0x0000000735200000, 0x0000000735400000, 0x0000000735400000|100%| E|CS|TAMS 0x0000000735200000, 0x0000000735200000| Complete -| 426|0x0000000735400000, 0x0000000735600000, 0x0000000735600000|100%| E|CS|TAMS 0x0000000735400000, 0x0000000735400000| Complete -| 427|0x0000000735600000, 0x0000000735800000, 0x0000000735800000|100%| E|CS|TAMS 0x0000000735600000, 0x0000000735600000| Complete -| 428|0x0000000735800000, 0x0000000735a00000, 0x0000000735a00000|100%| E|CS|TAMS 0x0000000735800000, 0x0000000735800000| Complete -| 429|0x0000000735a00000, 0x0000000735c00000, 0x0000000735c00000|100%| E|CS|TAMS 0x0000000735a00000, 0x0000000735a00000| Complete -| 430|0x0000000735c00000, 0x0000000735e00000, 0x0000000735e00000|100%| E|CS|TAMS 0x0000000735c00000, 0x0000000735c00000| Complete -| 431|0x0000000735e00000, 0x0000000736000000, 0x0000000736000000|100%| E|CS|TAMS 0x0000000735e00000, 0x0000000735e00000| Complete -| 432|0x0000000736000000, 0x0000000736200000, 0x0000000736200000|100%| E|CS|TAMS 0x0000000736000000, 0x0000000736000000| Complete -| 433|0x0000000736200000, 0x0000000736400000, 0x0000000736400000|100%| E|CS|TAMS 0x0000000736200000, 0x0000000736200000| Complete -| 434|0x0000000736400000, 0x0000000736600000, 0x0000000736600000|100%| E|CS|TAMS 0x0000000736400000, 0x0000000736400000| Complete -| 435|0x0000000736600000, 0x0000000736800000, 0x0000000736800000|100%| E|CS|TAMS 0x0000000736600000, 0x0000000736600000| Complete -| 436|0x0000000736800000, 0x0000000736a00000, 0x0000000736a00000|100%| E|CS|TAMS 0x0000000736800000, 0x0000000736800000| Complete -| 437|0x0000000736a00000, 0x0000000736c00000, 0x0000000736c00000|100%| E|CS|TAMS 0x0000000736a00000, 0x0000000736a00000| Complete -| 438|0x0000000736c00000, 0x0000000736e00000, 0x0000000736e00000|100%| E|CS|TAMS 0x0000000736c00000, 0x0000000736c00000| Complete -| 439|0x0000000736e00000, 0x0000000737000000, 0x0000000737000000|100%| E|CS|TAMS 0x0000000736e00000, 0x0000000736e00000| Complete -| 440|0x0000000737000000, 0x0000000737200000, 0x0000000737200000|100%| E|CS|TAMS 0x0000000737000000, 0x0000000737000000| Complete -| 441|0x0000000737200000, 0x0000000737400000, 0x0000000737400000|100%| E|CS|TAMS 0x0000000737200000, 0x0000000737200000| Complete -| 442|0x0000000737400000, 0x0000000737600000, 0x0000000737600000|100%| E|CS|TAMS 0x0000000737400000, 0x0000000737400000| Complete -| 443|0x0000000737600000, 0x0000000737800000, 0x0000000737800000|100%| E|CS|TAMS 0x0000000737600000, 0x0000000737600000| Complete -| 444|0x0000000737800000, 0x0000000737a00000, 0x0000000737a00000|100%| E|CS|TAMS 0x0000000737800000, 0x0000000737800000| Complete -| 445|0x0000000737a00000, 0x0000000737c00000, 0x0000000737c00000|100%| E|CS|TAMS 0x0000000737a00000, 0x0000000737a00000| Complete -| 446|0x0000000737c00000, 0x0000000737e00000, 0x0000000737e00000|100%| E|CS|TAMS 0x0000000737c00000, 0x0000000737c00000| Complete -| 447|0x0000000737e00000, 0x0000000738000000, 0x0000000738000000|100%| E|CS|TAMS 0x0000000737e00000, 0x0000000737e00000| Complete -| 448|0x0000000738000000, 0x0000000738200000, 0x0000000738200000|100%| E|CS|TAMS 0x0000000738000000, 0x0000000738000000| Complete -| 449|0x0000000738200000, 0x0000000738400000, 0x0000000738400000|100%| E|CS|TAMS 0x0000000738200000, 0x0000000738200000| Complete -| 450|0x0000000738400000, 0x0000000738600000, 0x0000000738600000|100%| E|CS|TAMS 0x0000000738400000, 0x0000000738400000| Complete -| 451|0x0000000738600000, 0x0000000738800000, 0x0000000738800000|100%| E|CS|TAMS 0x0000000738600000, 0x0000000738600000| Complete -| 452|0x0000000738800000, 0x0000000738a00000, 0x0000000738a00000|100%| E|CS|TAMS 0x0000000738800000, 0x0000000738800000| Complete -| 453|0x0000000738a00000, 0x0000000738c00000, 0x0000000738c00000|100%| E|CS|TAMS 0x0000000738a00000, 0x0000000738a00000| Complete -| 454|0x0000000738c00000, 0x0000000738e00000, 0x0000000738e00000|100%| E|CS|TAMS 0x0000000738c00000, 0x0000000738c00000| Complete -| 455|0x0000000738e00000, 0x0000000739000000, 0x0000000739000000|100%| E|CS|TAMS 0x0000000738e00000, 0x0000000738e00000| Complete -| 456|0x0000000739000000, 0x0000000739200000, 0x0000000739200000|100%| E|CS|TAMS 0x0000000739000000, 0x0000000739000000| Complete -| 457|0x0000000739200000, 0x0000000739400000, 0x0000000739400000|100%| E|CS|TAMS 0x0000000739200000, 0x0000000739200000| Complete -| 458|0x0000000739400000, 0x0000000739600000, 0x0000000739600000|100%| E|CS|TAMS 0x0000000739400000, 0x0000000739400000| Complete -| 459|0x0000000739600000, 0x0000000739800000, 0x0000000739800000|100%| E|CS|TAMS 0x0000000739600000, 0x0000000739600000| Complete -| 460|0x0000000739800000, 0x0000000739a00000, 0x0000000739a00000|100%| E|CS|TAMS 0x0000000739800000, 0x0000000739800000| Complete -| 461|0x0000000739a00000, 0x0000000739c00000, 0x0000000739c00000|100%| E|CS|TAMS 0x0000000739a00000, 0x0000000739a00000| Complete -| 462|0x0000000739c00000, 0x0000000739e00000, 0x0000000739e00000|100%| E|CS|TAMS 0x0000000739c00000, 0x0000000739c00000| Complete -| 767|0x000000075fe00000, 0x0000000760000000, 0x0000000760000000|100%| E|CS|TAMS 0x000000075fe00000, 0x000000075fe00000| Complete -|2046|0x00000007ffc00000, 0x00000007ffd74000, 0x00000007ffe00000| 72%|OA| |TAMS 0x00000007ffd74000, 0x00000007ffd74000| Untracked -|2047|0x00000007ffe00000, 0x00000007ffe74000, 0x0000000800000000| 22%|CA| |TAMS 0x00000007ffe74000, 0x00000007ffe74000| Untracked - -Card table byte_map: [0x0000000107fd4000,0x00000001087d4000] _byte_map_base: 0x00000001047d4000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x00000001033fb6c0, (CMBitMap*) 0x00000001033fb680 - Prev Bits: [0x0000000124b30000, 0x0000000128b30000) - Next Bits: [0x0000000120b30000, 0x0000000124b30000) - -Polling page: 0x0000000102b68000 - -Metaspace: - -Usage: - Non-class: 147.68 MB used. - Class: 18.98 MB used. - Both: 166.66 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 148.44 MB ( 77%) committed, 3 nodes. - Class space: 1.00 GB reserved, 19.62 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 168.06 MB ( 14%) committed. - -Chunk freelists: - Non-Class: 10.80 MB - Class: 12.42 MB - Both: 23.22 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 241.19 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 12. -num_arena_births: 2812. -num_arena_deaths: 434. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2686. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 620. -num_chunks_taken_from_freelist: 10025. -num_chunk_merges: 224. -num_chunk_splits: 6875. -num_chunks_enlarged: 4900. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=21056Kb max_used=21056Kb free=98111Kb - bounds [0x0000000118ed0000, 0x000000011a370000, 0x0000000120330000] -CodeHeap 'profiled nmethods': size=119152Kb used=55552Kb max_used=55552Kb free=63599Kb - bounds [0x0000000111330000, 0x0000000114980000, 0x000000011878c000] -CodeHeap 'non-nmethods': size=7440Kb used=3363Kb max_used=3558Kb free=4077Kb - bounds [0x000000011878c000, 0x0000000118b1c000, 0x0000000118ed0000] - total_blobs=28079 nmethods=26975 adapters=1014 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 44.759 Thread 0x0000000c688a7600 38501 4 org.apache.spark.serializer.JavaDeserializationStream$$anon$1:: (34 bytes) -Event: 44.762 Thread 0x0000000c654af600 38502 3 org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec::productIterator (18 bytes) -Event: 44.762 Thread 0x0000000c654af600 nmethod 38502 0x0000000114972f90 code [0x0000000114973140, 0x0000000114973338] -Event: 44.765 Thread 0x0000000c688a6400 38503 3 com.google.protobuf.TextFormatEscaper$1:: (18 bytes) -Event: 44.765 Thread 0x0000000c654aea00 38504 3 com.google.protobuf.TextFormatEscaper::escapeBytes (441 bytes) -Event: 44.765 Thread 0x0000000c688a6400 nmethod 38503 0x0000000114973490 code [0x0000000114973640, 0x0000000114973888] -Event: 44.765 Thread 0x0000000c654aea00 nmethod 38504 0x0000000114973a10 code [0x0000000114973f80, 0x00000001149758f8] -Event: 44.766 Thread 0x0000000c654af000 38505 3 org.apache.spark.sql.catalyst.trees.SQLQueryContext::isValid (176 bytes) -Event: 44.766 Thread 0x0000000c654af600 38506 3 org.apache.comet.serde.PartitioningOuterClass$Partitioning$Builder:: (21 bytes) -Event: 44.766 Thread 0x0000000c654aea00 38507 3 org.apache.comet.serde.OperatorOuterClass$ShuffleWriter$Builder::buildPartial (57 bytes) -Event: 44.766 Thread 0x0000000c688a6400 38508 3 scala.collection.immutable.MapLike$$anon$1::seq (13 bytes) -Event: 44.766 Thread 0x0000000c654af600 nmethod 38506 0x0000000114976390 code [0x0000000114976540, 0x0000000114976708] -Event: 44.766 Thread 0x0000000c654af600 38509 3 org.apache.spark.sql.comet.execution.shuffle.CometShuffleWriterInputIterator::nativeIterator (14 bytes) -Event: 44.766 Thread 0x0000000c688a6400 nmethod 38508 0x0000000114976810 code [0x00000001149769c0, 0x0000000114976bb8] -Event: 44.766 Thread 0x0000000c688a6400 38510 3 org.apache.comet.CometExecIterator::enableStashMode (16 bytes) -Event: 44.766 Thread 0x0000000c654aea00 nmethod 38507 0x0000000114976d10 code [0x0000000114976f00, 0x0000000114977328] -Event: 44.766 Thread 0x0000000c654af600 nmethod 38509 0x0000000114977490 code [0x0000000114977640, 0x00000001149777c8] -Event: 44.766 Thread 0x0000000c688a6400 nmethod 38510 0x0000000114977910 code [0x0000000114977ac0, 0x0000000114977cb8] -Event: 44.766 Thread 0x0000000c654af000 nmethod 38505 0x0000000114977e10 code [0x0000000114978240, 0x0000000114979588] -Event: 44.767 Thread 0x0000000c688a7600 nmethod 38501 0x000000011a35ba90 code [0x000000011a35be00, 0x000000011a35d1e0] - -GC Heap History (20 events): -Event: 43.099 GC heap before -{Heap before GC invocations=54 (full 0): - garbage-first heap total 954368K, used 427333K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 60 young (122880K), 5 survivors (10240K) - Metaspace used 168094K, committed 169472K, reserved 1245184K - class space used 19354K, committed 19968K, reserved 1048576K -} -Event: 43.102 GC heap after -{Heap after GC invocations=55 (full 0): - garbage-first heap total 954368K, used 136887K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 168094K, committed 169472K, reserved 1245184K - class space used 19354K, committed 19968K, reserved 1048576K -} -Event: 43.273 GC heap before -{Heap before GC invocations=55 (full 0): - garbage-first heap total 954368K, used 435895K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 65 young (133120K), 5 survivors (10240K) - Metaspace used 168345K, committed 169728K, reserved 1245184K - class space used 19362K, committed 19968K, reserved 1048576K -} -Event: 43.276 GC heap after -{Heap after GC invocations=56 (full 0): - garbage-first heap total 954368K, used 137390K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 168345K, committed 169728K, reserved 1245184K - class space used 19362K, committed 19968K, reserved 1048576K -} -Event: 43.452 GC heap before -{Heap before GC invocations=56 (full 0): - garbage-first heap total 954368K, used 438446K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 65 young (133120K), 5 survivors (10240K) - Metaspace used 168649K, committed 170048K, reserved 1245184K - class space used 19365K, committed 20032K, reserved 1048576K -} -Event: 43.455 GC heap after -{Heap after GC invocations=57 (full 0): - garbage-first heap total 954368K, used 138096K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 5 young (10240K), 5 survivors (10240K) - Metaspace used 168649K, committed 170048K, reserved 1245184K - class space used 19365K, committed 20032K, reserved 1048576K -} -Event: 43.648 GC heap before -{Heap before GC invocations=57 (full 0): - garbage-first heap total 954368K, used 424816K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 60 young (122880K), 5 survivors (10240K) - Metaspace used 169313K, committed 170688K, reserved 1245184K - class space used 19405K, committed 20032K, reserved 1048576K -} -Event: 43.650 GC heap after -{Heap after GC invocations=58 (full 0): - garbage-first heap total 954368K, used 138744K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 169313K, committed 170688K, reserved 1245184K - class space used 19405K, committed 20032K, reserved 1048576K -} -Event: 43.793 GC heap before -{Heap before GC invocations=58 (full 0): - garbage-first heap total 954368K, used 417272K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 54 young (110592K), 4 survivors (8192K) - Metaspace used 169487K, committed 170816K, reserved 1245184K - class space used 19406K, committed 20032K, reserved 1048576K -} -Event: 43.795 GC heap after -{Heap after GC invocations=59 (full 0): - garbage-first heap total 954368K, used 138412K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 169487K, committed 170816K, reserved 1245184K - class space used 19406K, committed 20032K, reserved 1048576K -} -Event: 44.025 GC heap before -{Heap before GC invocations=59 (full 0): - garbage-first heap total 954368K, used 449708K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 72 young (147456K), 4 survivors (8192K) - Metaspace used 169922K, committed 171328K, reserved 1245184K - class space used 19436K, committed 20096K, reserved 1048576K -} -Event: 44.027 GC heap after -{Heap after GC invocations=60 (full 0): - garbage-first heap total 954368K, used 138976K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 169922K, committed 171328K, reserved 1245184K - class space used 19436K, committed 20096K, reserved 1048576K -} -Event: 44.170 GC heap before -{Heap before GC invocations=60 (full 0): - garbage-first heap total 954368K, used 427744K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 60 young (122880K), 4 survivors (8192K) - Metaspace used 170050K, committed 171456K, reserved 1245184K - class space used 19436K, committed 20096K, reserved 1048576K -} -Event: 44.172 GC heap after -{Heap after GC invocations=61 (full 0): - garbage-first heap total 954368K, used 139147K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 170050K, committed 171456K, reserved 1245184K - class space used 19436K, committed 20096K, reserved 1048576K -} -Event: 44.310 GC heap before -{Heap before GC invocations=61 (full 0): - garbage-first heap total 954368K, used 423819K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 57 young (116736K), 4 survivors (8192K) - Metaspace used 170204K, committed 171648K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} -Event: 44.312 GC heap after -{Heap after GC invocations=62 (full 0): - garbage-first heap total 954368K, used 139345K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 170204K, committed 171648K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} -Event: 44.459 GC heap before -{Heap before GC invocations=62 (full 0): - garbage-first heap total 954368K, used 417873K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 56 young (114688K), 4 survivors (8192K) - Metaspace used 170279K, committed 171712K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} -Event: 44.461 GC heap after -{Heap after GC invocations=63 (full 0): - garbage-first heap total 954368K, used 138883K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 170279K, committed 171712K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} -Event: 44.618 GC heap before -{Heap before GC invocations=63 (full 0): - garbage-first heap total 954368K, used 433795K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 62 young (126976K), 4 survivors (8192K) - Metaspace used 170506K, committed 171904K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} -Event: 44.620 GC heap after -{Heap after GC invocations=64 (full 0): - garbage-first heap total 954368K, used 139704K [0x0000000700000000, 0x0000000800000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 170506K, committed 171904K, reserved 1245184K - class space used 19439K, committed 20096K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.379 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.379 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.344 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-1214774261196833962.dylib -Event: 5.837 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 5.837 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 5.837 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 5.837 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 5.837 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 5.837 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 5.837 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.837 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 5.837 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 5.837 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.837 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 5.837 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 5.837 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 5.837 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 5.981 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-3296482769080348166.dylib -Event: 6.743 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-9c7a5abf-870c-4741-a8f7-18e9355385fb-libsnappyjava.dylib -Event: 39.824 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/jnilib-13936019976582104019.tmp - -Deoptimization events (20 events): -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000119ff7358 relative=0x0000000000000218 -Event: 44.435 Thread 0x0000000c67a99e00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000119ff7358 method=org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory$IntegerUpdater.readValue(ILorg/apache/spark/sql/execution/vectorized/WritableColumnVector; -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT PACKING pc=0x0000000119ff7358 sp=0x000000017de54fa0 -Event: 44.435 Thread 0x0000000c67a99e00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017de54f10 mode 2 -Event: 44.482 Thread 0x0000000c67a9aa00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00000001195069b8 relative=0x0000000000001138 -Event: 44.482 Thread 0x0000000c67a9aa00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00000001195069b8 method=java.util.concurrent.ConcurrentHashMap.putVal(Ljava/lang/Object;Ljava/lang/Object;Z)Ljava/lang/Object; @ 162 c2 -Event: 44.482 Thread 0x0000000c67a9aa00 DEOPT PACKING pc=0x00000001195069b8 sp=0x000000017e66db70 -Event: 44.482 Thread 0x0000000c67a9aa00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017e66daa0 mode 2 -Event: 44.486 Thread 0x0000000c67a9aa00 Uncommon trap: trap_request=0xffffff45 fr.pc=0x000000011943b6c8 relative=0x0000000000000388 -Event: 44.486 Thread 0x0000000c67a9aa00 Uncommon trap: reason=unstable_if action=reinterpret pc=0x000000011943b6c8 method=java.util.concurrent.ConcurrentHashMap.replaceNode(Ljava/lang/Object;Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object; @ 150 c2 -Event: 44.486 Thread 0x0000000c67a9aa00 DEOPT PACKING pc=0x000000011943b6c8 sp=0x000000017e66dca0 -Event: 44.486 Thread 0x0000000c67a9aa00 DEOPT UNPACKING pc=0x00000001187d391c sp=0x000000017e66dbe0 mode 2 - -Classes loaded (20 events): -Event: 42.562 Loading class jdk/internal/reflect/UnsafeQualifiedStaticIntegerFieldAccessorImpl -Event: 42.562 Loading class jdk/internal/reflect/UnsafeQualifiedStaticIntegerFieldAccessorImpl done -Event: 42.724 Loading class java/nio/channels/Channels$WritableByteChannelImpl -Event: 42.724 Loading class java/nio/channels/Channels$WritableByteChannelImpl done -Event: 42.889 Loading class java/lang/org -Event: 42.889 Loading class java/lang/org done -Event: 43.056 Loading class java/lang/org -Event: 43.056 Loading class java/lang/org done -Event: 43.217 Loading class java/lang/org -Event: 43.217 Loading class java/lang/org done -Event: 43.377 Loading class java/lang/org -Event: 43.377 Loading class java/lang/org done -Event: 43.937 Loading class java/util/stream/StreamSpliterators$WrappingSpliterator -Event: 43.937 Loading class java/util/stream/StreamSpliterators$AbstractWrappingSpliterator -Event: 43.937 Loading class java/util/stream/StreamSpliterators$AbstractWrappingSpliterator done -Event: 43.937 Loading class java/util/stream/StreamSpliterators$WrappingSpliterator done -Event: 43.937 Loading class java/util/stream/StreamSpliterators -Event: 43.938 Loading class java/util/stream/StreamSpliterators done -Event: 43.938 Loading class java/util/function/BooleanSupplier -Event: 43.938 Loading class java/util/function/BooleanSupplier done - -Classes unloaded (20 events): -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1b000 'java/lang/invoke/LambdaForm$MH+0x000000c801e1b000' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1ac00 'java/lang/invoke/LambdaForm$MH+0x000000c801e1ac00' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1a800 'java/lang/invoke/LambdaForm$MH+0x000000c801e1a800' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e1a400 'java/lang/invoke/LambdaForm$MH+0x000000c801e1a400' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19800 'java/lang/invoke/LambdaForm$MH+0x000000c801e19800' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19400 'java/lang/invoke/LambdaForm$MH+0x000000c801e19400' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e19000 'java/lang/invoke/LambdaForm$MH+0x000000c801e19000' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e18c00' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18400 'java/lang/invoke/LambdaForm$MH+0x000000c801e18400' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e18000 'java/lang/invoke/LambdaForm$MH+0x000000c801e18000' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e16c00' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16800 'java/lang/invoke/LambdaForm$MH+0x000000c801e16800' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e16000 'java/lang/invoke/LambdaForm$MH+0x000000c801e16000' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e15c00' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15800 'java/lang/invoke/LambdaForm$MH+0x000000c801e15800' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e15400 'java/lang/invoke/LambdaForm$MH+0x000000c801e15400' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14c00 'java/lang/invoke/LambdaForm$MH+0x000000c801e14c00' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14400 'java/lang/invoke/LambdaForm$MH+0x000000c801e14400' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14000 'java/lang/invoke/LambdaForm$MH+0x000000c801e14000' -Event: 7.444 Thread 0x0000000103401c20 Unloading class 0x000000c801e14800 'java/lang/invoke/LambdaForm$MH+0x000000c801e14800' - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 40.510 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736dc4b30) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a34d90) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a444e8) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a47f10) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.511 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736a56cc0) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368cc910) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368e3068) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x00000007368f62e0) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.516 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736909fb8) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.517 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000736935678) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.547 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 -Event: 40.583 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 -Event: 40.584 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 -Event: 40.584 Thread 0x00000001033f4320 Implicit null exception at 0x0000000119a5f5a0 to 0x0000000119a5fdc0 -Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b4bc58) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b55800) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.781 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737b86b70) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 40.782 Thread 0x0000000c66d1f000 Exception ()V> (0x0000000737baa740) -thrown [src/hotspot/share/prims/jni.cpp, line 1112] -Event: 42.152 Thread 0x0000000c64289e00 Implicit null exception at 0x0000000119c003d4 to 0x0000000119c00cb4 -Event: 43.939 Thread 0x00000001033f4320 Implicit null exception at 0x0000000118fb6e18 to 0x0000000118fb6f50 - -VM Operations (20 events): -Event: 43.793 Executing VM operation: G1TryInitiateConcMark -Event: 43.795 Executing VM operation: G1TryInitiateConcMark done -Event: 43.951 Executing VM operation: ICBufferFull -Event: 43.951 Executing VM operation: ICBufferFull done -Event: 44.007 Executing VM operation: ICBufferFull -Event: 44.007 Executing VM operation: ICBufferFull done -Event: 44.025 Executing VM operation: G1TryInitiateConcMark -Event: 44.027 Executing VM operation: G1TryInitiateConcMark done -Event: 44.170 Executing VM operation: G1TryInitiateConcMark -Event: 44.172 Executing VM operation: G1TryInitiateConcMark done -Event: 44.310 Executing VM operation: G1TryInitiateConcMark -Event: 44.312 Executing VM operation: G1TryInitiateConcMark done -Event: 44.312 Executing VM operation: G1TryInitiateConcMark -Event: 44.312 Executing VM operation: G1TryInitiateConcMark done -Event: 44.459 Executing VM operation: G1TryInitiateConcMark -Event: 44.461 Executing VM operation: G1TryInitiateConcMark done -Event: 44.618 Executing VM operation: G1TryInitiateConcMark -Event: 44.620 Executing VM operation: G1TryInitiateConcMark done -Event: 44.620 Executing VM operation: G1TryInitiateConcMark -Event: 44.620 Executing VM operation: G1TryInitiateConcMark done - -Memory protections (20 events): -Event: 44.408 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 -Event: 44.408 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 -Event: 44.409 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 -Event: 44.409 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 -Event: 44.515 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 -Event: 44.515 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 -Event: 44.515 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 -Event: 44.515 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 -Event: 44.549 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 -Event: 44.549 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 -Event: 44.549 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 -Event: 44.549 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 -Event: 44.655 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 -Event: 44.655 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 -Event: 44.655 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 -Event: 44.655 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 -Event: 44.687 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 0 -Event: 44.688 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 0 -Event: 44.688 Protecting memory [0x0000000338efc000,0x0000000338f08000] with protection modes 3 -Event: 44.688 Protecting memory [0x0000000338af0000,0x0000000338afc000] with protection modes 3 - -Nmethod flushes (20 events): -Event: 43.577 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001129c0490 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bd9110 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bdbb90 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bebf90 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bec490 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bec990 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112bf2c90 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112cab890 -Event: 43.578 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112cf9e90 -Event: 43.579 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000112ef4b90 -Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bb810 -Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bbf10 -Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131bcd90 -Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131c6690 -Event: 43.580 Thread 0x0000000c688a6a00 flushing nmethod 0x00000001131c7510 -Event: 43.582 Thread 0x0000000c688a6a00 flushing osr nmethod 0x0000000113970910 -Event: 43.582 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000113bcc690 -Event: 43.582 Thread 0x0000000c688a6a00 flushing nmethod 0x0000000113bccb10 -Event: 43.582 Thread 0x0000000c688a6a00 flushing osr nmethod 0x0000000113bdd910 -Event: 43.585 Thread 0x0000000c688a6a00 flushing nmethod 0x000000011414f590 - -Events (20 events): -Event: 44.408 Thread 0x0000000c67a99e00 Thread added: 0x0000000c638b1800 -Event: 44.408 Thread 0x0000000c638b1800 Thread added: 0x0000000c638b1e00 -Event: 44.409 Thread 0x0000000c638b1e00 Thread exited: 0x0000000c638b1e00 -Event: 44.409 Thread 0x0000000c638b1800 Thread exited: 0x0000000c638b1800 -Event: 44.515 Thread 0x00000001033f4320 Thread added: 0x0000000c638b3600 -Event: 44.515 Thread 0x0000000c638b3600 Thread added: 0x0000000c638d0000 -Event: 44.515 Thread 0x0000000c638d0000 Thread exited: 0x0000000c638d0000 -Event: 44.516 Thread 0x0000000c638b3600 Thread exited: 0x0000000c638b3600 -Event: 44.549 Thread 0x0000000c67a9a400 Thread added: 0x0000000c638b3600 -Event: 44.549 Thread 0x0000000c638b3600 Thread added: 0x0000000c638d0000 -Event: 44.549 Thread 0x0000000c638d0000 Thread exited: 0x0000000c638d0000 -Event: 44.549 Thread 0x0000000c638b3600 Thread exited: 0x0000000c638b3600 -Event: 44.655 Thread 0x00000001033f4320 Thread added: 0x0000000c638d0c00 -Event: 44.655 Thread 0x0000000c638d0c00 Thread added: 0x0000000c638d1200 -Event: 44.655 Thread 0x0000000c638d1200 Thread exited: 0x0000000c638d1200 -Event: 44.655 Thread 0x0000000c638d0c00 Thread exited: 0x0000000c638d0c00 -Event: 44.687 Thread 0x0000000c67a9a400 Thread added: 0x0000000c638d0c00 -Event: 44.688 Thread 0x0000000c638d0c00 Thread added: 0x0000000c638d1200 -Event: 44.688 Thread 0x0000000c638d1200 Thread exited: 0x0000000c638d1200 -Event: 44.688 Thread 0x0000000c638d0c00 Thread exited: 0x0000000c638d0c00 - - -Dynamic libraries: -0x0000000102b80000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x00000001087ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x0000000102ba8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x0000000102bd8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x0000000102c28000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x0000000102e9c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x0000000102ec4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x0000000102ee4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x0000000102f08000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x0000000102e88000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x0000000102f24000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x0000000102f38000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x000000012acd8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000151e04000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-1214774261196833962.dylib -0x000000012ad20000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-3296482769080348166.dylib -0x000000012ad4c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-9c7a5abf-870c-4741-a8f7-18e9355385fb-libsnappyjava.dylib -0x0000000102e74000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/jnilib-13936019976582104019.tmp - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:41 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 6.34 7.11 5.42 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(3840832k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. diff --git a/spark/hs_err_pid59323.log b/spark/hs_err_pid59323.log deleted file mode 100644 index 9411333aa6..0000000000 --- a/spark/hs_err_pid59323.log +++ /dev/null @@ -1,1626 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x0000000108449f64, pid=59323, tid=64019 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 08:59:57 2026 MDT elapsed time: 21.549643 seconds (0d 0h 0m 21s) - ---------------- T H R E A D --------------- - -Current thread (0x000000093d9d8600): JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x000000017e688000,0x000000017ea8b000)] - -Stack: [0x000000017e688000,0x000000017ea8b000], sp=0x000000017ea72120, free space=4008k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bdf64] jni_IsInstanceOf+0x15c -C [libcomet-6348563280641709946.dylib+0x697ce4c] jni::env::Env::is_instance_of_class::hbf417139dcae8a78+0x2a0 -C [libcomet-6348563280641709946.dylib+0x75a74] jni::env::Env::is_instance_of::hf7a3ed7ac336cf21+0x114 -C [libcomet-6348563280641709946.dylib+0x36e098] comet::execution::planner::PhysicalPlanner::create_plan::_$u7b$$u7b$closure$u7d$$u7d$::h66151afd53abe41e+0x5c -C [libcomet-6348563280641709946.dylib+0x191528] datafusion_comet_jni_bridge::JVMClasses::with_env::hc5494511a77e45ae+0x1f8 -C [libcomet-6348563280641709946.dylib+0x3357fc] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0xd94 -C [libcomet-6348563280641709946.dylib+0x3388a8] comet::execution::planner::PhysicalPlanner::create_plan::ha97acc115968b76f+0x3e40 -C [libcomet-6348563280641709946.dylib+0x18f48] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0x1c4 -C [libcomet-6348563280641709946.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-6348563280641709946.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-6348563280641709946.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-6348563280641709946.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-6348563280641709946.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-6348563280641709946.dylib+0x106628] __rust_try+0x20 -C [libcomet-6348563280641709946.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-6348563280641709946.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-6348563280641709946.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-6348563280641709946.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3997+0x000000e0021c93d0.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3988+0x000000e0021c8000.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2525+0x000000e001df5ed0.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3997+0x000000e0021c93d0.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3988+0x000000e0021c8000.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2525+0x000000e001df5ed0.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0xba9cde559ff17bfe - -Registers: - x0=0x000000e0021b54c0 x1=0x0000000000000010 x2=0x000000094432ea20 x3=0x000000017ea77298 - x4=0x0000000000000238 x5=0x0000000000000018 x6=0x000000094b9050e0 x7=0xfffff0003ffff800 - x8=0x00000000021b54c0 x9=0x0000000000000000 x10=0x000000e000000000 x11=0x000000093f17d3c0 -x12=0x0000000942256490 x13=0x0000000000000001 x14=0x00000001027d5a08 x15=0x000000093f17c000 -x16=0x000000018c8009d0 x17=0x000000093d839400 x18=0x0000000000000000 x19=0x000000093d9d8600 -x20=0x0000000000000001 x21=0x0000000102bae3a0 x22=0xba9cde559ff17bea x23=0x0000000108ba9088 -x24=0x0000000000000000 x25=0x0000000108badf38 x26=0x0000000136fd7a48 x27=0x00000004d7d9d260 -x28=0x000000093d9d8600 fp=0x000000017ea72160 lr=0x0000000108449f38 sp=0x000000017ea72120 -pc=0x0000000108449f64 cpsr=0x0000000040001000 - -Register to memory mapping: - - x0=0x000000e0021b54c0 is a pointer to class: -org.apache.comet.CometHandleBatchIterator {0x000000e0021b54c0} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x00000001370504c8) - - method ordering: Array(0x000000e000913f08) - - local interfaces: Array(0x000000e0005d4fb8) - - trans. interfaces: Array(0x000000e0005d4fb8) - - constants: constant pool [62]/operands[4] {0x000000013704bd98} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000137050708 - - class loader data: loader data: 0x000000093f007d40 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x000000e0005d4f48) - - nest members: Array(0x000000e0005d4f48) - - permitted subclasses: Array(0x000000e0005d4f48) - - java mirror: a 'java/lang/Class'{0x00000004e01635c0} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x000000e0021b5690) - - itable length 2 (start addr: 0x000000e0021b56c0) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x1=0x0000000000000010 is an unknown value - x2=0x000000094432ea20 points into unknown readable memory: 0x00000004ff8bdfa8 | a8 df 8b ff 04 00 00 00 - x3=0x000000017ea77298 is pointing into the stack for thread: 0x000000093d9d8600 - x4=0x0000000000000238 is an unknown value - x5=0x0000000000000018 is an unknown value - x6=0x000000094b9050e0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=35345600 is a compressed pointer to class: 0x000000e0021b54c0 -org.apache.comet.CometHandleBatchIterator {0x000000e0021b54c0} - - instance size: 2 - - klass size: 67 - - access: public synchronized - - state: fully_initialized - - name: 'org/apache/comet/CometHandleBatchIterator' - - super: 'java/lang/Object' - - sub: - - arrays: NULL - - methods: Array(0x00000001370504c8) - - method ordering: Array(0x000000e000913f08) - - local interfaces: Array(0x000000e0005d4fb8) - - trans. interfaces: Array(0x000000e0005d4fb8) - - constants: constant pool [62]/operands[4] {0x000000013704bd98} for 'org/apache/comet/CometHandleBatchIterator' cache=0x0000000137050708 - - class loader data: loader data: 0x000000093f007d40 for instance a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} - - source file: 'CometHandleBatchIterator.java' - - class annotations: Array(0x0000000000000000) - - class type annotations: Array(0x0000000000000000) - - field annotations: Array(0x0000000000000000) - - field type annotations: Array(0x0000000000000000) - - inner classes: Array(0x000000e0005d4f48) - - nest members: Array(0x000000e0005d4f48) - - permitted subclasses: Array(0x000000e0005d4f48) - - java mirror: a 'java/lang/Class'{0x00000004e01635c0} = 'org/apache/comet/CometHandleBatchIterator' - - vtable length 6 (start addr: 0x000000e0021b5690) - - itable length 2 (start addr: 0x000000e0021b56c0) - - ---- static fields (0 words): - - ---- non-static fields (1 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 - - non-static oop maps: 12-12 - x9=0x0 is NULL -x10=0x000000e000000000 is pointing into metadata -x11=0x000000093f17d3c0 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -x12=0x0000000942256490 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x13=0x0000000000000001 is an unknown value -x14=0x00000001027d5a08 points into unknown readable memory: 0x000fd05c80438aa8 | a8 8a 43 80 5c d0 0f 00 -x15=0x000000093f17c000 points into unknown readable memory: 0xffffffffffffffff | ff ff ff ff ff ff ff ff -x16=0x000000018c8009d0: pthread_jit_write_protect_np+0 in /usr/lib/system/libsystem_pthread.dylib at 0x000000018c7f9000 -x17=0x000000093d839400 points into unknown readable memory: 0x0000000000000000 | 00 00 00 00 00 00 00 00 -x18=0x0 is NULL -x19=0x000000093d9d8600 is a thread -x20=0x0000000000000001 is an unknown value -x21=0x0000000102bae3a0 is a global jni handle -x22=0xba9cde559ff17bea is an unknown value -x23=0x0000000108ba9088: _ZN14AccessInternal15RuntimeDispatchILy548932EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 -x24=0x0 is NULL -x25=0x0000000108badf38: _ZN14AccessInternal15RuntimeDispatchILy598084EP7oopDescLNS_11BarrierTypeE2EE10_load_funcE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 -x26=0x0000000136fd7a48 is pointing into metadata -x27=0x00000004d7d9d260 is an oop: java.lang.Class -{0x00000004d7d9d260} - klass: 'java/lang/Class' - - ---- fields (total size 15 words): - - private volatile transient 'classRedefinedCount' 'I' @12 0 - - private volatile transient 'cachedConstructor' 'Ljava/lang/reflect/Constructor;' @40 NULL (0) - - private transient 'name' 'Ljava/lang/String;' @44 "scala.collection.AbstractSeq"{0x00000004d7d9d2d8} (9afb3a5b) - - private transient 'module' 'Ljava/lang/Module;' @48 a 'java/lang/Module'{0x00000004d7aedbf8} (9af5db7f) - - private final 'classLoader' 'Ljava/lang/ClassLoader;' @52 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000004d7aeb770} (9af5d6ee) - - private transient 'classData' 'Ljava/lang/Object;' @56 NULL (0) - - private transient 'packageName' 'Ljava/lang/String;' @60 "scala.collection"{0x00000004d7aec308} (9af5d861) - - private final 'componentType' 'Ljava/lang/Class;' @64 NULL (0) - - private volatile transient 'reflectionData' 'Ljava/lang/ref/SoftReference;' @68 a 'java/lang/ref/SoftReference'{0x00000004dc05e488} (9b80bc91) - - private volatile transient 'genericInfo' 'Lsun/reflect/generics/repository/ClassRepository;' @72 NULL (0) - - private volatile transient 'enumConstants' '[Ljava/lang/Object;' @76 NULL (0) - - private volatile transient 'enumConstantDirectory' 'Ljava/util/Map;' @80 NULL (0) - - private volatile transient 'annotationData' 'Ljava/lang/Class$AnnotationData;' @84 NULL (0) - - private volatile transient 'annotationType' 'Lsun/reflect/annotation/AnnotationType;' @88 NULL (0) - - transient 'classValueMap' 'Ljava/lang/ClassValue$ClassValueMap;' @92 NULL (0) - - signature: Lscala/collection/AbstractSeq; - - fake entry for mirror: 'scala/collection/AbstractSeq' - - fake entry for array: 'scala/collection/AbstractSeq'[] - - fake entry for oop_size: 15 - - fake entry for static_oop_field_count: 1 - - private static transient synthetic '$jacocoData' '[Z' @112 [Z{0x00000004d8669470} (9b0cd28e) -x28=0x000000093d9d8600 is a thread - - -Top of Stack: (sp=0x000000017ea72120) -0x000000017ea72120: 0000000136fd7a48 000000009c5e19cd -0x000000017ea72130: 000000017ea89818 00000004e2f0d270 -0x000000017ea72140: 0000000000000000 0000000108bf7f90 -0x000000017ea72150: 000000017ea89738 0000000123a6772c -0x000000017ea72160: 000000017ea725f0 000000014e74ce4c -0x000000017ea72170: 0000000000000000 000000015166e880 -0x000000017ea72180: 000000017ea72190 000000014e786d54 -0x000000017ea72190: 000000094432ea20 0000000102bae3a0 -0x000000017ea721a0: 0000000108449e08 000000093f194b90 -0x000000017ea721b0: 0000000108bb2ad0 0000000108bb2ad0 -0x000000017ea721c0: 000000093d9d88b0 000000093d9d88b0 -0x000000017ea721d0: 000000015167ae90 000000004e74ee14 -0x000000017ea721e0: 0000000000000000 000000017ea72270 -0x000000017ea721f0: 000000093d9d88b0 000000015167ae90 -0x000000017ea72200: 000000003d9d8600 000000017ea72860 -0x000000017ea72210: 000000017ea72a00 8000000000000020 -0x000000017ea72220: 000000015167ae90 000000093f194b90 -0x000000017ea72230: 000000015167ae90 8000000000000020 -0x000000017ea72240: 000000015167ae90 000000093d9d88b0 -0x000000017ea72250: 000000093d9d88b0 0000000000000002 -0x000000017ea72260: 000000093d839400 0000000000000000 -0x000000017ea72270: 000000093d9d88b0 0000000000000002 -0x000000017ea72280: 000000093d839400 8000000000000020 -0x000000017ea72290: 000000015167ae90 000000093d8394d0 -0x000000017ea722a0: 000000017ea722e0 000000014e7afc28 -0x000000017ea722b0: 0000000000000000 0000000000000000 -0x000000017ea722c0: 000000093d9d88b0 000000015166e888 -0x000000017ea722d0: 0000000000000000 000000093d9d88b0 -0x000000017ea722e0: 000000093d9d88b0 0000000000000002 -0x000000017ea722f0: 0000000000000008 000000015166e888 -0x000000017ea72300: 000000017ea72340 000000014e787d8c -0x000000017ea72310: 000000017ea723f8 000000093d9d88b0 - -Instructions: (pc=0x0000000108449f64) -0x0000000108449e64: 7f 82 02 b9 00 00 80 52 d0 c7 0b 94 18 00 80 52 -0x0000000108449e74: 03 00 00 14 14 00 80 52 38 00 80 52 68 e2 0c 91 -0x0000000108449e84: a9 00 80 52 09 fd 9f 88 bf 3b 03 d5 68 02 0d 91 -0x0000000108449e94: 08 c1 bf f8 88 00 00 36 e0 03 13 aa 21 00 80 52 -0x0000000108449ea4: 7e fd 0d 94 68 2e 43 b9 88 00 00 35 68 2a 43 b9 -0x0000000108449eb4: 1f 05 1e 72 80 00 00 54 e0 03 13 aa 01 00 80 52 -0x0000000108449ec4: fc aa 13 94 68 e2 0c 91 c9 00 80 52 09 fd 9f 88 -0x0000000108449ed4: e0 03 17 aa e1 03 15 aa e2 03 16 aa 1f 20 03 d5 -0x0000000108449ee4: 35 06 00 b4 17 3b 00 90 f7 22 02 91 c0 fa 7f 92 -0x0000000108449ef4: 39 3b 00 90 39 e3 3c 91 df 02 40 f2 e8 02 99 9a -0x0000000108449f04: 08 01 40 f9 00 01 3f d6 c8 3c 00 d0 08 41 35 91 -0x0000000108449f14: 01 01 40 b9 9e 77 0b 94 80 03 00 b4 f6 03 00 aa -0x0000000108449f24: a0 fa 7f 92 bf 02 40 f2 e8 02 99 9a 08 01 40 f9 -0x0000000108449f34: 00 01 3f d6 c8 3c 00 d0 08 65 0a 91 08 01 40 39 -0x0000000108449f44: 88 02 00 34 08 08 40 b9 29 3b 00 b0 29 41 24 91 -0x0000000108449f54: 2a 01 40 f9 29 09 40 b9 08 21 c9 9a 00 01 0a 8b -0x0000000108449f64: c8 16 40 b9 09 68 68 f8 3f 01 16 eb c0 01 00 54 -0x0000000108449f74: 1f 81 00 71 a1 00 00 54 e1 03 16 aa 5e 07 06 94 -0x0000000108449f84: f5 03 00 aa 09 00 00 14 15 00 80 52 07 00 00 14 -0x0000000108449f94: 00 04 40 f9 c8 16 40 b9 09 68 68 f8 3f 01 16 eb -0x0000000108449fa4: 81 fe ff 54 35 00 80 52 e0 03 15 2a 1f 20 03 d5 -0x0000000108449fb4: 76 76 40 f9 c8 0a 40 f9 09 01 40 f9 89 00 00 b4 -0x0000000108449fc4: e0 03 16 aa d7 f9 fc 97 c8 0a 40 f9 c9 06 40 f9 -0x0000000108449fd4: 28 09 00 f9 c8 0e 40 f9 c9 06 40 f9 28 0d 00 f9 -0x0000000108449fe4: c8 12 40 f9 c9 06 40 f9 28 11 00 f9 60 42 0a 91 -0x0000000108449ff4: 63 1f fa 97 bf 3b 03 d5 68 e2 0c 91 89 00 80 52 -0x000000010844a004: 09 fd 9f 88 f8 00 00 37 68 82 42 b9 1f 01 14 6b -0x000000010844a014: 80 00 00 54 74 82 02 b9 e0 03 14 aa 63 c7 0b 94 -0x000000010844a024: e0 03 15 aa fd 7b 44 a9 f4 4f 43 a9 f6 57 42 a9 -0x000000010844a034: f8 5f 41 a9 fa 67 c5 a8 c0 03 5f d6 ff c3 01 d1 -0x000000010844a044: fa 67 02 a9 f8 5f 03 a9 f6 57 04 a9 f4 4f 05 a9 -0x000000010844a054: fd 7b 06 a9 fd 83 01 91 f5 03 03 aa f6 03 02 aa - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x0000000136fd7a48 is pointing into metadata -stack at sp + 1 slots: 2623412685 is a compressed pointer to object: [B -{0x00000004e2f0ce68} - klass: {type array byte} - - length: 50 -stack at sp + 2 slots: 0x000000017ea89818 is pointing into the stack for thread: 0x000000093d9d8600 -stack at sp + 3 slots: 0x00000004e2f0d270 is an oop: scala.Tuple2 -{0x00000004e2f0d270} - klass: 'scala/Tuple2' - - ---- fields (total size 3 words): - - public final '_1' 'Ljava/lang/Object;' @12 a 'org/apache/arrow/c/ArrowArray'[1] {0x00000004e2f0cf28} (9c5e19e5) - - public final '_2' 'Ljava/lang/Object;' @16 a 'org/apache/arrow/c/ArrowSchema'[1] {0x00000004e2f0cf40} (9c5e19e8) -stack at sp + 4 slots: 0x0 is NULL -stack at sp + 5 slots: 0x0000000108bf7f90: _ZN19TemplateInterpreter13_active_tableE+0 in /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib at 0x0000000107f8c000 -stack at sp + 6 slots: 0x000000017ea89738 is pointing into the stack for thread: 0x000000093d9d8600 -stack at sp + 7 slots: 0x0000000123a6772c is at code_begin+44 in an Interpreter codelet -result handlers for native calls [0x0000000123a67700, 0x0000000123a67780] 128 bytes - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x00000009461f2900, length=73, elements={ -0x0000000102b94610, 0x000000093e870000, 0x000000093e870600, 0x000000093e871200, -0x000000093e871800, 0x000000093e871e00, 0x000000093e872400, 0x000000093e872a00, -0x000000093e873000, 0x000000093e873600, 0x000000093e8daa00, 0x000000093dddaa00, -0x000000093e6c9e00, 0x000000093e6cb000, 0x000000093e6cb600, 0x000000093e738000, -0x000000093e738600, 0x000000093e738c00, 0x000000093e73b000, 0x000000093e73b600, -0x000000093de9c000, 0x000000093de9c600, 0x000000093de9cc00, 0x000000093de9d200, -0x000000093de9d800, 0x000000093de9de00, 0x000000093e750600, 0x000000093e752400, -0x000000093e752a00, 0x000000093e532a00, 0x000000093e784000, 0x000000093e784c00, -0x000000093e785200, 0x000000093e785800, 0x000000093e785e00, 0x000000093e786a00, -0x000000093e674c00, 0x000000093e674600, 0x000000093e675200, 0x000000093e533000, -0x000000093de29800, 0x000000093dd41800, 0x000000093e7e4000, 0x000000093e7f4600, -0x000000093e7f5e00, 0x000000093e7f6a00, 0x000000093d07d200, 0x000000093df77000, -0x000000093d9bf600, 0x000000093d9d8000, 0x000000093d9d8600, 0x000000093d9d8c00, -0x000000093d9d9200, 0x000000093d9ed200, 0x000000093d9ede00, 0x000000093d9ee400, -0x000000093d9ef000, 0x000000093d156400, 0x000000093e8db600, 0x000000093d9d9e00, -0x000000093d96b600, 0x000000093db16400, 0x000000093db17000, 0x000000093db17600, -0x000000093db68000, 0x000000093db68600, 0x000000093dbba400, 0x000000093dbbaa00, -0x000000093dbbb000, 0x000000093dbbb600, 0x000000093e8d8000, 0x000000093db6b600, -0x000000093b2acc00 -} - -Java Threads: ( => current thread ) - 0x0000000102b94610 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4355, stack(0x000000016db74000,0x000000016df77000)] - 0x000000093e870000 JavaThread "Reference Handler" daemon [_thread_blocked, id=23555, stack(0x000000016ebc8000,0x000000016efcb000)] - 0x000000093e870600 JavaThread "Finalizer" daemon [_thread_blocked, id=31747, stack(0x000000016efd4000,0x000000016f3d7000)] - 0x000000093e871200 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30467, stack(0x000000016f4f8000,0x000000016f8fb000)] - 0x000000093e871800 JavaThread "Service Thread" daemon [_thread_blocked, id=29955, stack(0x000000016f904000,0x000000016fd07000)] - 0x000000093e871e00 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=26371, stack(0x000000016fd10000,0x0000000170113000)] - 0x000000093e872400 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=29699, stack(0x000000017011c000,0x000000017031f000)] - 0x000000093e872a00 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=29187, stack(0x0000000170328000,0x000000017052b000)] - 0x000000093e873000 JavaThread "Sweeper thread" daemon [_thread_blocked, id=28931, stack(0x0000000170534000,0x0000000170937000)] - 0x000000093e873600 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=27395, stack(0x0000000170940000,0x0000000170d43000)] - 0x000000093e8daa00 JavaThread "Notification Thread" daemon [_thread_blocked, id=42755, stack(0x0000000171788000,0x0000000171b8b000)] - 0x000000093dddaa00 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=44035, stack(0x0000000174484000,0x0000000174887000)] - 0x000000093e6c9e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=61699, stack(0x00000001754d8000,0x00000001758db000)] - 0x000000093e6cb000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=45315, stack(0x00000001758e4000,0x0000000175ce7000)] - 0x000000093e6cb600 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=61443, stack(0x0000000175cf0000,0x00000001760f3000)] - 0x000000093e738000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=60931, stack(0x00000001760fc000,0x00000001764ff000)] - 0x000000093e738600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=46083, stack(0x0000000176508000,0x000000017690b000)] - 0x000000093e738c00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=46339, stack(0x0000000176914000,0x0000000176d17000)] - 0x000000093e73b000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=46595, stack(0x0000000176d20000,0x0000000177123000)] - 0x000000093e73b600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=46851, stack(0x000000017712c000,0x000000017752f000)] - 0x000000093de9c000 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=59651, stack(0x0000000177538000,0x000000017793b000)] - 0x000000093de9c600 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=47107, stack(0x0000000177944000,0x0000000177d47000)] - 0x000000093de9cc00 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=59139, stack(0x0000000177d50000,0x0000000178153000)] - 0x000000093de9d200 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=47875, stack(0x000000017815c000,0x000000017855f000)] - 0x000000093de9d800 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=48387, stack(0x0000000178568000,0x000000017896b000)] - 0x000000093de9de00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=48899, stack(0x0000000178974000,0x0000000178d77000)] - 0x000000093e750600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=49155, stack(0x0000000178d80000,0x0000000179183000)] - 0x000000093e752400 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=58371, stack(0x000000017918c000,0x000000017958f000)] - 0x000000093e752a00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=49411, stack(0x0000000179598000,0x000000017999b000)] - 0x000000093e532a00 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=57603, stack(0x00000001799a4000,0x0000000179da7000)] - 0x000000093e784000 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=49923, stack(0x0000000179db0000,0x000000017a1b3000)] - 0x000000093e784c00 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=57091, stack(0x000000017a1bc000,0x000000017a5bf000)] - 0x000000093e785200 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=50435, stack(0x000000017a5c8000,0x000000017a9cb000)] - 0x000000093e785800 JavaThread "task-abort-timer" daemon [_thread_blocked, id=56835, stack(0x000000017a9d4000,0x000000017add7000)] - 0x000000093e785e00 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=50947, stack(0x000000017ade0000,0x000000017b1e3000)] - 0x000000093e786a00 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=56067, stack(0x000000017b1ec000,0x000000017b5ef000)] - 0x000000093e674c00 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=51203, stack(0x000000017b5f8000,0x000000017b9fb000)] - 0x000000093e674600 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=51459, stack(0x000000017ba04000,0x000000017be07000)] - 0x000000093e675200 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=51715, stack(0x000000017be10000,0x000000017c213000)] - 0x000000093e533000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=55043, stack(0x000000017c21c000,0x000000017c61f000)] - 0x000000093de29800 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=54531, stack(0x000000017c628000,0x000000017ca2b000)] - 0x000000093dd41800 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=52227, stack(0x000000017ca34000,0x000000017ce37000)] - 0x000000093e7e4000 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=53763, stack(0x000000017ce40000,0x000000017d243000)] - 0x000000093e7f4600 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=52739, stack(0x000000017d24c000,0x000000017d64f000)] - 0x000000093e7f5e00 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=52995, stack(0x000000017d658000,0x000000017da5b000)] - 0x000000093e7f6a00 JavaThread "Timer-0" [_thread_blocked, id=65539, stack(0x000000017da64000,0x000000017de67000)] - 0x000000093d07d200 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=86787, stack(0x000000017de70000,0x000000017e273000)] - 0x000000093df77000 JavaThread "process reaper" daemon [_thread_blocked, id=33551, stack(0x0000000174890000,0x00000001748c7000)] - 0x000000093d9bf600 JavaThread "Executor task launch worker for task 1.0 in stage 4.0 (TID 13)" daemon [_thread_in_vm, id=43287, stack(0x0000000171370000,0x0000000171773000)] - 0x000000093d9d8000 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_vm, id=33315, stack(0x000000017e27c000,0x000000017e67f000)] -=>0x000000093d9d8600 JavaThread "Executor task launch worker for task 0.0 in stage 4.0 (TID 12)" daemon [_thread_in_vm, id=64019, stack(0x000000017e688000,0x000000017ea8b000)] - 0x000000093d9d8c00 JavaThread "Executor task launch worker for task 3.0 in stage 4.0 (TID 15)" daemon [_thread_in_vm, id=30995, stack(0x000000017ea94000,0x000000017ee97000)] - 0x000000093d9d9200 JavaThread "Executor task launch worker for task 2.0 in stage 4.0 (TID 14)" daemon [_thread_in_vm, id=86275, stack(0x000000017eea0000,0x000000017f2a3000)] - 0x000000093d9ed200 JavaThread "process reaper" daemon [_thread_blocked, id=85251, stack(0x000000016f46c000,0x000000016f4a3000)] - 0x000000093d9ede00 JavaThread "process reaper" daemon [_thread_blocked, id=66051, stack(0x000000016f4ac000,0x000000016f4e3000)] - 0x000000093d9ee400 JavaThread "process reaper" daemon [_thread_blocked, id=66307, stack(0x0000000174adc000,0x0000000174b13000)] - 0x000000093d9ef000 JavaThread "process reaper" daemon [_thread_blocked, id=83203, stack(0x0000000174b1c000,0x0000000174b53000)] - 0x000000093d156400 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=84787, stack(0x000000017f2ac000,0x000000017f6af000)] - 0x000000093e8db600 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=67379, stack(0x000000017f6b8000,0x000000017fabb000)] - 0x000000093d9d9e00 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=84035, stack(0x000000017fac4000,0x000000017fec7000)] - 0x000000093d96b600 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=82483, stack(0x0000000300004000,0x0000000300407000)] - 0x000000093db16400 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=82803, stack(0x0000000300410000,0x0000000300813000)] - 0x000000093db17000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=67103, stack(0x000000030081c000,0x0000000300c1f000)] - 0x000000093db17600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=81667, stack(0x0000000300c28000,0x000000030102b000)] - 0x000000093db68000 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81411, stack(0x0000000301034000,0x0000000301437000)] - 0x000000093db68600 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=80899, stack(0x0000000301440000,0x0000000301843000)] - 0x000000093dbba400 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=28679, stack(0x0000000170d4c000,0x0000000170f4f000)] - 0x000000093dbbaa00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=27919, stack(0x0000000170f58000,0x000000017115b000)] - 0x000000093dbbb000 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=69907, stack(0x0000000171164000,0x0000000171367000)] - 0x000000093dbbb600 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=69643, stack(0x00000001748d0000,0x0000000174ad3000)] - 0x000000093e8d8000 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=28179, stack(0x0000000302288000,0x000000030248b000)] - 0x000000093db6b600 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=85511, stack(0x00000003026a0000,0x0000000302aa3000)] - 0x000000093b2acc00 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=33039, stack(0x0000000302aac000,0x0000000302caf000)] - -Other Threads: - 0x0000000102ba64e0 VMThread "VM Thread" [stack: 0x000000016e9bc000,0x000000016ebbf000] [id=19971] - 0x000000093e3e4300 WatcherThread [stack: 0x0000000171b94000,0x0000000171d97000] [id=42243] - 0x0000000102b9e340 GCTaskThread "GC Thread#0" [stack: 0x000000016df80000,0x000000016e183000] [id=14595] - 0x000000093ddbcf00 GCTaskThread "GC Thread#1" [stack: 0x0000000171da0000,0x0000000171fa3000] [id=41731] - 0x000000093ddbd200 GCTaskThread "GC Thread#2" [stack: 0x0000000171fac000,0x00000001721af000] [id=41219] - 0x000000093ddbd500 GCTaskThread "GC Thread#3" [stack: 0x00000001721b8000,0x00000001723bb000] [id=40963] - 0x000000093ddbd800 GCTaskThread "GC Thread#4" [stack: 0x00000001723c4000,0x00000001725c7000] [id=40707] - 0x000000093ddbdb00 GCTaskThread "GC Thread#5" [stack: 0x00000001725d0000,0x00000001727d3000] [id=34563] - 0x000000093ddbde00 GCTaskThread "GC Thread#6" [stack: 0x00000001727dc000,0x00000001729df000] [id=35075] - 0x000000093ddbe100 GCTaskThread "GC Thread#7" [stack: 0x00000001729e8000,0x0000000172beb000] [id=40195] - 0x000000093ddbe400 GCTaskThread "GC Thread#8" [stack: 0x0000000172bf4000,0x0000000172df7000] [id=39683] - 0x000000093ddbe700 GCTaskThread "GC Thread#9" [stack: 0x0000000172e00000,0x0000000173003000] [id=35587] - 0x000000093ddbea00 GCTaskThread "GC Thread#10" [stack: 0x000000017300c000,0x000000017320f000] [id=36099] - 0x000000093ddbed00 GCTaskThread "GC Thread#11" [stack: 0x0000000173218000,0x000000017341b000] [id=39171] - 0x000000093ddbf000 GCTaskThread "GC Thread#12" [stack: 0x0000000173424000,0x0000000173627000] [id=38659] - 0x000000093ddbf300 GCTaskThread "GC Thread#13" [stack: 0x0000000173630000,0x0000000173833000] [id=38147] - 0x000000093ddbf600 GCTaskThread "GC Thread#14" [stack: 0x000000017383c000,0x0000000173a3f000] [id=37635] - 0x000000093ddbf900 GCTaskThread "GC Thread#15" [stack: 0x0000000173a48000,0x0000000173c4b000] [id=37123] - 0x000000093ddbfc00 GCTaskThread "GC Thread#16" [stack: 0x0000000173c54000,0x0000000173e57000] [id=36611] - 0x000000093e4e8000 GCTaskThread "GC Thread#17" [stack: 0x0000000173e60000,0x0000000174063000] [id=43523] - 0x000000093e4e8300 GCTaskThread "GC Thread#18" [stack: 0x000000017406c000,0x000000017426f000] [id=65283] - 0x000000093e4e8600 GCTaskThread "GC Thread#19" [stack: 0x0000000174278000,0x000000017447b000] [id=64771] - 0x0000000102b9f030 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016e18c000,0x000000016e38f000] [id=12803] - 0x0000000102b9f9a0 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016e398000,0x000000016e59b000] [id=13827] - 0x000000093e3e5b00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174ca8000,0x0000000174eab000] [id=63235] - 0x000000093e3e4900 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000174eb4000,0x00000001750b7000] [id=62723] - 0x000000093e3e4600 ConcurrentGCThread "G1 Conc#3" [stack: 0x00000001750c0000,0x00000001752c3000] [id=44803] - 0x000000093ddbc600 ConcurrentGCThread "G1 Conc#4" [stack: 0x00000001752cc000,0x00000001754cf000] [id=62211] - 0x0000000102ba1a60 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016e5a4000,0x000000016e7a7000] [id=16643] - 0x0000000102ba2360 ConcurrentGCThread "G1 Service" [stack: 0x000000016e7b0000,0x000000016e9b3000] [id=17155] - -Threads with active compile tasks: - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x00000004d5000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x000000e000000000-0x000000e000bc0000-0x000000e000bc0000), size 12320768, SharedBaseAddress: 0x000000e000000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x000000e001000000-0x000000e041000000, reserved size: 1073741824 -Narrow klass base: 0x000000e000000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 342016K, used 264885K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 59 young (120832K), 4 survivors (8192K) - Metaspace used 151478K, committed 152768K, reserved 1245184K - class space used 18229K, committed 18816K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x00000004d5000000, 0x00000004d5200000, 0x00000004d5200000|100%|HS| |TAMS 0x00000004d5200000, 0x00000004d5000000| Complete -| 1|0x00000004d5200000, 0x00000004d5400000, 0x00000004d5400000|100%|HS| |TAMS 0x00000004d5400000, 0x00000004d5200000| Complete -| 2|0x00000004d5400000, 0x00000004d5600000, 0x00000004d5600000|100%|HC| |TAMS 0x00000004d5600000, 0x00000004d5400000| Complete -| 3|0x00000004d5600000, 0x00000004d5800000, 0x00000004d5800000|100%|HS| |TAMS 0x00000004d5800000, 0x00000004d5600000| Complete -| 4|0x00000004d5800000, 0x00000004d5a00000, 0x00000004d5a00000|100%|HC| |TAMS 0x00000004d5a00000, 0x00000004d5800000| Complete -| 5|0x00000004d5a00000, 0x00000004d5c00000, 0x00000004d5c00000|100%|HC| |TAMS 0x00000004d5c00000, 0x00000004d5a00000| Complete -| 6|0x00000004d5c00000, 0x00000004d5e00000, 0x00000004d5e00000|100%|HC| |TAMS 0x00000004d5e00000, 0x00000004d5c00000| Complete -| 7|0x00000004d5e00000, 0x00000004d6000000, 0x00000004d6000000|100%|HC| |TAMS 0x00000004d6000000, 0x00000004d5e00000| Complete -| 8|0x00000004d6000000, 0x00000004d6200000, 0x00000004d6200000|100%|HC| |TAMS 0x00000004d6200000, 0x00000004d6000000| Complete -| 9|0x00000004d6200000, 0x00000004d6400000, 0x00000004d6400000|100%|HC| |TAMS 0x00000004d6400000, 0x00000004d6200000| Complete -| 10|0x00000004d6400000, 0x00000004d6600000, 0x00000004d6600000|100%|HC| |TAMS 0x00000004d6600000, 0x00000004d6400000| Complete -| 11|0x00000004d6600000, 0x00000004d6800000, 0x00000004d6800000|100%|HS| |TAMS 0x00000004d6800000, 0x00000004d6600000| Complete -| 12|0x00000004d6800000, 0x00000004d6a00000, 0x00000004d6a00000|100%|HS| |TAMS 0x00000004d6a00000, 0x00000004d6800000| Complete -| 13|0x00000004d6a00000, 0x00000004d6c00000, 0x00000004d6c00000|100%|HC| |TAMS 0x00000004d6c00000, 0x00000004d6a00000| Complete -| 14|0x00000004d6c00000, 0x00000004d6e00000, 0x00000004d6e00000|100%|HS| |TAMS 0x00000004d6e00000, 0x00000004d6c00000| Complete -| 15|0x00000004d6e00000, 0x00000004d7000000, 0x00000004d7000000|100%|HC| |TAMS 0x00000004d7000000, 0x00000004d6e00000| Complete -| 16|0x00000004d7000000, 0x00000004d7200000, 0x00000004d7200000|100%|HS| |TAMS 0x00000004d7200000, 0x00000004d7000000| Complete -| 17|0x00000004d7200000, 0x00000004d7400000, 0x00000004d7400000|100%|HS| |TAMS 0x00000004d7400000, 0x00000004d7200000| Complete -| 18|0x00000004d7400000, 0x00000004d7600000, 0x00000004d7600000|100%|HC| |TAMS 0x00000004d7600000, 0x00000004d7400000| Complete -| 19|0x00000004d7600000, 0x00000004d7800000, 0x00000004d7800000|100%| O| |TAMS 0x00000004d7800000, 0x00000004d7600000| Untracked -| 20|0x00000004d7800000, 0x00000004d7a00000, 0x00000004d7a00000|100%| O| |TAMS 0x00000004d7a00000, 0x00000004d7800000| Untracked -| 21|0x00000004d7a00000, 0x00000004d7c00000, 0x00000004d7c00000|100%| O| |TAMS 0x00000004d7c00000, 0x00000004d7a00000| Untracked -| 22|0x00000004d7c00000, 0x00000004d7e00000, 0x00000004d7e00000|100%| O| |TAMS 0x00000004d7e00000, 0x00000004d7c00000| Untracked -| 23|0x00000004d7e00000, 0x00000004d8000000, 0x00000004d8000000|100%| O| |TAMS 0x00000004d8000000, 0x00000004d7e00000| Untracked -| 24|0x00000004d8000000, 0x00000004d8200000, 0x00000004d8200000|100%| O| |TAMS 0x00000004d8200000, 0x00000004d8000000| Untracked -| 25|0x00000004d8200000, 0x00000004d8400000, 0x00000004d8400000|100%| O| |TAMS 0x00000004d8400000, 0x00000004d8200000| Untracked -| 26|0x00000004d8400000, 0x00000004d8600000, 0x00000004d8600000|100%| O| |TAMS 0x00000004d8600000, 0x00000004d8400000| Untracked -| 27|0x00000004d8600000, 0x00000004d8800000, 0x00000004d8800000|100%| O| |TAMS 0x00000004d8800000, 0x00000004d8600000| Untracked -| 28|0x00000004d8800000, 0x00000004d8a00000, 0x00000004d8a00000|100%| O| |TAMS 0x00000004d8a00000, 0x00000004d8800000| Untracked -| 29|0x00000004d8a00000, 0x00000004d8c00000, 0x00000004d8c00000|100%| O| |TAMS 0x00000004d8c00000, 0x00000004d8a00000| Untracked -| 30|0x00000004d8c00000, 0x00000004d8e00000, 0x00000004d8e00000|100%| O| |TAMS 0x00000004d8e00000, 0x00000004d8c00000| Untracked -| 31|0x00000004d8e00000, 0x00000004d9000000, 0x00000004d9000000|100%| O| |TAMS 0x00000004d9000000, 0x00000004d8e00000| Untracked -| 32|0x00000004d9000000, 0x00000004d9200000, 0x00000004d9200000|100%| O| |TAMS 0x00000004d9200000, 0x00000004d9000000| Untracked -| 33|0x00000004d9200000, 0x00000004d9400000, 0x00000004d9400000|100%| O| |TAMS 0x00000004d9400000, 0x00000004d9200000| Untracked -| 34|0x00000004d9400000, 0x00000004d9600000, 0x00000004d9600000|100%| O| |TAMS 0x00000004d9600000, 0x00000004d9400000| Untracked -| 35|0x00000004d9600000, 0x00000004d9800000, 0x00000004d9800000|100%| O| |TAMS 0x00000004d9800000, 0x00000004d9600000| Untracked -| 36|0x00000004d9800000, 0x00000004d9a00000, 0x00000004d9a00000|100%| O| |TAMS 0x00000004d9a00000, 0x00000004d9800000| Untracked -| 37|0x00000004d9a00000, 0x00000004d9c00000, 0x00000004d9c00000|100%| O| |TAMS 0x00000004d9c00000, 0x00000004d9a00000| Untracked -| 38|0x00000004d9c00000, 0x00000004d9e00000, 0x00000004d9e00000|100%| O| |TAMS 0x00000004d9e00000, 0x00000004d9c00000| Untracked -| 39|0x00000004d9e00000, 0x00000004da000000, 0x00000004da000000|100%| O| |TAMS 0x00000004da000000, 0x00000004d9e00000| Untracked -| 40|0x00000004da000000, 0x00000004da200000, 0x00000004da200000|100%| O| |TAMS 0x00000004da200000, 0x00000004da000000| Untracked -| 41|0x00000004da200000, 0x00000004da3c8200, 0x00000004da400000| 89%| O| |TAMS 0x00000004da3c8200, 0x00000004da200000| Untracked -| 42|0x00000004da400000, 0x00000004da600000, 0x00000004da600000|100%|HS| |TAMS 0x00000004da400000, 0x00000004da400000| Complete -| 43|0x00000004da600000, 0x00000004da800000, 0x00000004da800000|100%|HS| |TAMS 0x00000004da600000, 0x00000004da600000| Complete -| 44|0x00000004da800000, 0x00000004daa00000, 0x00000004daa00000|100%|HS| |TAMS 0x00000004da800000, 0x00000004da800000| Complete -| 45|0x00000004daa00000, 0x00000004dac00000, 0x00000004dac00000|100%|HS| |TAMS 0x00000004daa00000, 0x00000004daa00000| Complete -| 46|0x00000004dac00000, 0x00000004dae00000, 0x00000004dae00000|100%|HS| |TAMS 0x00000004dac00000, 0x00000004dac00000| Complete -| 47|0x00000004dae00000, 0x00000004db000000, 0x00000004db000000|100%|HS| |TAMS 0x00000004dae00000, 0x00000004dae00000| Complete -| 48|0x00000004db000000, 0x00000004db200000, 0x00000004db200000|100%|HS| |TAMS 0x00000004db000000, 0x00000004db000000| Complete -| 49|0x00000004db200000, 0x00000004db400000, 0x00000004db400000|100%| O| |TAMS 0x00000004db200000, 0x00000004db200000| Untracked -| 50|0x00000004db400000, 0x00000004db600000, 0x00000004db600000|100%| O| |TAMS 0x00000004db400000, 0x00000004db400000| Untracked -| 51|0x00000004db600000, 0x00000004db800000, 0x00000004db800000|100%|HS| |TAMS 0x00000004db600000, 0x00000004db600000| Complete -| 52|0x00000004db800000, 0x00000004dba00000, 0x00000004dba00000|100%|HS| |TAMS 0x00000004db800000, 0x00000004db800000| Complete -| 53|0x00000004dba00000, 0x00000004dbc00000, 0x00000004dbc00000|100%|HS| |TAMS 0x00000004dba00000, 0x00000004dba00000| Complete -| 54|0x00000004dbc00000, 0x00000004dbe00000, 0x00000004dbe00000|100%| O| |TAMS 0x00000004dbc00000, 0x00000004dbc00000| Untracked -| 55|0x00000004dbe00000, 0x00000004dc000000, 0x00000004dc000000|100%| O| |TAMS 0x00000004dbe00000, 0x00000004dbe00000| Untracked -| 56|0x00000004dc000000, 0x00000004dc200000, 0x00000004dc200000|100%| O| |TAMS 0x00000004dc000000, 0x00000004dc000000| Untracked -| 57|0x00000004dc200000, 0x00000004dc400000, 0x00000004dc400000|100%| O| |TAMS 0x00000004dc200000, 0x00000004dc200000| Untracked -| 58|0x00000004dc400000, 0x00000004dc600000, 0x00000004dc600000|100%|HS| |TAMS 0x00000004dc400000, 0x00000004dc400000| Complete -| 59|0x00000004dc600000, 0x00000004dc800000, 0x00000004dc800000|100%|HC| |TAMS 0x00000004dc600000, 0x00000004dc600000| Complete -| 60|0x00000004dc800000, 0x00000004dca00000, 0x00000004dca00000|100%|HC| |TAMS 0x00000004dc800000, 0x00000004dc800000| Complete -| 61|0x00000004dca00000, 0x00000004dca00000, 0x00000004dcc00000| 0%| F| |TAMS 0x00000004dca00000, 0x00000004dca00000| Untracked -| 62|0x00000004dcc00000, 0x00000004dcc00000, 0x00000004dce00000| 0%| F| |TAMS 0x00000004dcc00000, 0x00000004dcc00000| Untracked -| 63|0x00000004dce00000, 0x00000004dd000000, 0x00000004dd000000|100%| O| |TAMS 0x00000004dce00000, 0x00000004dce00000| Untracked -| 64|0x00000004dd000000, 0x00000004dd200000, 0x00000004dd200000|100%| O| |TAMS 0x00000004dd000000, 0x00000004dd000000| Untracked -| 65|0x00000004dd200000, 0x00000004dd291800, 0x00000004dd400000| 28%| O| |TAMS 0x00000004dd200000, 0x00000004dd200000| Untracked -| 66|0x00000004dd400000, 0x00000004dd400000, 0x00000004dd600000| 0%| F| |TAMS 0x00000004dd400000, 0x00000004dd400000| Untracked -| 67|0x00000004dd600000, 0x00000004dd600000, 0x00000004dd800000| 0%| F| |TAMS 0x00000004dd600000, 0x00000004dd600000| Untracked -| 68|0x00000004dd800000, 0x00000004dd800000, 0x00000004dda00000| 0%| F| |TAMS 0x00000004dd800000, 0x00000004dd800000| Untracked -| 69|0x00000004dda00000, 0x00000004dda00000, 0x00000004ddc00000| 0%| F| |TAMS 0x00000004dda00000, 0x00000004dda00000| Untracked -| 70|0x00000004ddc00000, 0x00000004ddc00000, 0x00000004dde00000| 0%| F| |TAMS 0x00000004ddc00000, 0x00000004ddc00000| Untracked -| 71|0x00000004dde00000, 0x00000004de000000, 0x00000004de000000|100%| O| |TAMS 0x00000004dde00000, 0x00000004dde00000| Untracked -| 72|0x00000004de000000, 0x00000004de200000, 0x00000004de200000|100%| O| |TAMS 0x00000004de000000, 0x00000004de000000| Untracked -| 73|0x00000004de200000, 0x00000004de400000, 0x00000004de400000|100%| O| |TAMS 0x00000004de200000, 0x00000004de200000| Untracked -| 74|0x00000004de400000, 0x00000004de600000, 0x00000004de600000|100%| O| |TAMS 0x00000004de400000, 0x00000004de400000| Untracked -| 75|0x00000004de600000, 0x00000004de800000, 0x00000004de800000|100%| O| |TAMS 0x00000004de600000, 0x00000004de600000| Untracked -| 76|0x00000004de800000, 0x00000004dea00000, 0x00000004dea00000|100%| O| |TAMS 0x00000004de800000, 0x00000004de800000| Untracked -| 77|0x00000004dea00000, 0x00000004dec00000, 0x00000004dec00000|100%| O| |TAMS 0x00000004dea00000, 0x00000004dea00000| Untracked -| 78|0x00000004dec00000, 0x00000004dee00000, 0x00000004dee00000|100%| O| |TAMS 0x00000004dec00000, 0x00000004dec00000| Untracked -| 79|0x00000004dee00000, 0x00000004dee00000, 0x00000004df000000| 0%| F| |TAMS 0x00000004dee00000, 0x00000004dee00000| Untracked -| 80|0x00000004df000000, 0x00000004df000000, 0x00000004df200000| 0%| F| |TAMS 0x00000004df000000, 0x00000004df000000| Untracked -| 81|0x00000004df200000, 0x00000004df200000, 0x00000004df400000| 0%| F| |TAMS 0x00000004df200000, 0x00000004df200000| Untracked -| 82|0x00000004df400000, 0x00000004df46bbf8, 0x00000004df600000| 21%| S|CS|TAMS 0x00000004df400000, 0x00000004df400000| Complete -| 83|0x00000004df600000, 0x00000004df600000, 0x00000004df800000| 0%| F| |TAMS 0x00000004df600000, 0x00000004df600000| Untracked -| 84|0x00000004df800000, 0x00000004df800000, 0x00000004dfa00000| 0%| F| |TAMS 0x00000004df800000, 0x00000004df800000| Untracked -| 85|0x00000004dfa00000, 0x00000004dfa00000, 0x00000004dfc00000| 0%| F| |TAMS 0x00000004dfa00000, 0x00000004dfa00000| Untracked -| 86|0x00000004dfc00000, 0x00000004dfe00000, 0x00000004dfe00000|100%| S|CS|TAMS 0x00000004dfc00000, 0x00000004dfc00000| Complete -| 87|0x00000004dfe00000, 0x00000004e0000000, 0x00000004e0000000|100%| S|CS|TAMS 0x00000004dfe00000, 0x00000004dfe00000| Complete -| 88|0x00000004e0000000, 0x00000004e0200000, 0x00000004e0200000|100%| S|CS|TAMS 0x00000004e0000000, 0x00000004e0000000| Complete -| 89|0x00000004e0200000, 0x00000004e0200000, 0x00000004e0400000| 0%| F| |TAMS 0x00000004e0200000, 0x00000004e0200000| Untracked -| 90|0x00000004e0400000, 0x00000004e0400000, 0x00000004e0600000| 0%| F| |TAMS 0x00000004e0400000, 0x00000004e0400000| Untracked -| 91|0x00000004e0600000, 0x00000004e0600000, 0x00000004e0800000| 0%| F| |TAMS 0x00000004e0600000, 0x00000004e0600000| Untracked -| 92|0x00000004e0800000, 0x00000004e0800000, 0x00000004e0a00000| 0%| F| |TAMS 0x00000004e0800000, 0x00000004e0800000| Untracked -| 93|0x00000004e0a00000, 0x00000004e0a00000, 0x00000004e0c00000| 0%| F| |TAMS 0x00000004e0a00000, 0x00000004e0a00000| Untracked -| 94|0x00000004e0c00000, 0x00000004e0c00000, 0x00000004e0e00000| 0%| F| |TAMS 0x00000004e0c00000, 0x00000004e0c00000| Untracked -| 95|0x00000004e0e00000, 0x00000004e0e00000, 0x00000004e1000000| 0%| F| |TAMS 0x00000004e0e00000, 0x00000004e0e00000| Untracked -| 96|0x00000004e1000000, 0x00000004e1000000, 0x00000004e1200000| 0%| F| |TAMS 0x00000004e1000000, 0x00000004e1000000| Untracked -| 97|0x00000004e1200000, 0x00000004e1200000, 0x00000004e1400000| 0%| F| |TAMS 0x00000004e1200000, 0x00000004e1200000| Untracked -| 98|0x00000004e1400000, 0x00000004e1400000, 0x00000004e1600000| 0%| F| |TAMS 0x00000004e1400000, 0x00000004e1400000| Untracked -| 99|0x00000004e1600000, 0x00000004e1600000, 0x00000004e1800000| 0%| F| |TAMS 0x00000004e1600000, 0x00000004e1600000| Untracked -| 100|0x00000004e1800000, 0x00000004e1800000, 0x00000004e1a00000| 0%| F| |TAMS 0x00000004e1800000, 0x00000004e1800000| Untracked -| 101|0x00000004e1a00000, 0x00000004e1a00000, 0x00000004e1c00000| 0%| F| |TAMS 0x00000004e1a00000, 0x00000004e1a00000| Untracked -| 102|0x00000004e1c00000, 0x00000004e1c00000, 0x00000004e1e00000| 0%| F| |TAMS 0x00000004e1c00000, 0x00000004e1c00000| Untracked -| 103|0x00000004e1e00000, 0x00000004e1e00000, 0x00000004e2000000| 0%| F| |TAMS 0x00000004e1e00000, 0x00000004e1e00000| Untracked -| 104|0x00000004e2000000, 0x00000004e2000000, 0x00000004e2200000| 0%| F| |TAMS 0x00000004e2000000, 0x00000004e2000000| Untracked -| 105|0x00000004e2200000, 0x00000004e2200000, 0x00000004e2400000| 0%| F| |TAMS 0x00000004e2200000, 0x00000004e2200000| Untracked -| 106|0x00000004e2400000, 0x00000004e2400000, 0x00000004e2600000| 0%| F| |TAMS 0x00000004e2400000, 0x00000004e2400000| Untracked -| 107|0x00000004e2600000, 0x00000004e2600000, 0x00000004e2800000| 0%| F| |TAMS 0x00000004e2600000, 0x00000004e2600000| Untracked -| 108|0x00000004e2800000, 0x00000004e2800000, 0x00000004e2a00000| 0%| F| |TAMS 0x00000004e2800000, 0x00000004e2800000| Untracked -| 109|0x00000004e2a00000, 0x00000004e2a00000, 0x00000004e2c00000| 0%| F| |TAMS 0x00000004e2a00000, 0x00000004e2a00000| Untracked -| 110|0x00000004e2c00000, 0x00000004e2d3c128, 0x00000004e2e00000| 61%| E| |TAMS 0x00000004e2c00000, 0x00000004e2c00000| Complete -| 111|0x00000004e2e00000, 0x00000004e3000000, 0x00000004e3000000|100%| E|CS|TAMS 0x00000004e2e00000, 0x00000004e2e00000| Complete -| 112|0x00000004e3000000, 0x00000004e3200000, 0x00000004e3200000|100%| E|CS|TAMS 0x00000004e3000000, 0x00000004e3000000| Complete -| 113|0x00000004e3200000, 0x00000004e3400000, 0x00000004e3400000|100%| E|CS|TAMS 0x00000004e3200000, 0x00000004e3200000| Complete -| 114|0x00000004e3400000, 0x00000004e3600000, 0x00000004e3600000|100%| E|CS|TAMS 0x00000004e3400000, 0x00000004e3400000| Complete -| 115|0x00000004e3600000, 0x00000004e3800000, 0x00000004e3800000|100%| E|CS|TAMS 0x00000004e3600000, 0x00000004e3600000| Complete -| 116|0x00000004e3800000, 0x00000004e3a00000, 0x00000004e3a00000|100%| E|CS|TAMS 0x00000004e3800000, 0x00000004e3800000| Complete -| 117|0x00000004e3a00000, 0x00000004e3c00000, 0x00000004e3c00000|100%| E|CS|TAMS 0x00000004e3a00000, 0x00000004e3a00000| Complete -| 118|0x00000004e3c00000, 0x00000004e3e00000, 0x00000004e3e00000|100%| E|CS|TAMS 0x00000004e3c00000, 0x00000004e3c00000| Complete -| 119|0x00000004e3e00000, 0x00000004e4000000, 0x00000004e4000000|100%| E|CS|TAMS 0x00000004e3e00000, 0x00000004e3e00000| Complete -| 120|0x00000004e4000000, 0x00000004e4200000, 0x00000004e4200000|100%| E|CS|TAMS 0x00000004e4000000, 0x00000004e4000000| Complete -| 121|0x00000004e4200000, 0x00000004e4400000, 0x00000004e4400000|100%| E|CS|TAMS 0x00000004e4200000, 0x00000004e4200000| Complete -| 122|0x00000004e4400000, 0x00000004e4600000, 0x00000004e4600000|100%| E|CS|TAMS 0x00000004e4400000, 0x00000004e4400000| Complete -| 123|0x00000004e4600000, 0x00000004e4800000, 0x00000004e4800000|100%| E|CS|TAMS 0x00000004e4600000, 0x00000004e4600000| Complete -| 124|0x00000004e4800000, 0x00000004e4a00000, 0x00000004e4a00000|100%| E|CS|TAMS 0x00000004e4800000, 0x00000004e4800000| Complete -| 125|0x00000004e4a00000, 0x00000004e4c00000, 0x00000004e4c00000|100%| E|CS|TAMS 0x00000004e4a00000, 0x00000004e4a00000| Complete -| 126|0x00000004e4c00000, 0x00000004e4e00000, 0x00000004e4e00000|100%| E|CS|TAMS 0x00000004e4c00000, 0x00000004e4c00000| Complete -| 127|0x00000004e4e00000, 0x00000004e5000000, 0x00000004e5000000|100%| E|CS|TAMS 0x00000004e4e00000, 0x00000004e4e00000| Complete -| 128|0x00000004e5000000, 0x00000004e5200000, 0x00000004e5200000|100%| E|CS|TAMS 0x00000004e5000000, 0x00000004e5000000| Complete -| 129|0x00000004e5200000, 0x00000004e5400000, 0x00000004e5400000|100%| E|CS|TAMS 0x00000004e5200000, 0x00000004e5200000| Complete -| 130|0x00000004e5400000, 0x00000004e5600000, 0x00000004e5600000|100%| E|CS|TAMS 0x00000004e5400000, 0x00000004e5400000| Complete -| 131|0x00000004e5600000, 0x00000004e5800000, 0x00000004e5800000|100%| E|CS|TAMS 0x00000004e5600000, 0x00000004e5600000| Complete -| 132|0x00000004e5800000, 0x00000004e5a00000, 0x00000004e5a00000|100%| E|CS|TAMS 0x00000004e5800000, 0x00000004e5800000| Complete -| 133|0x00000004e5a00000, 0x00000004e5c00000, 0x00000004e5c00000|100%| E|CS|TAMS 0x00000004e5a00000, 0x00000004e5a00000| Complete -| 134|0x00000004e5c00000, 0x00000004e5e00000, 0x00000004e5e00000|100%| E|CS|TAMS 0x00000004e5c00000, 0x00000004e5c00000| Complete -| 135|0x00000004e5e00000, 0x00000004e6000000, 0x00000004e6000000|100%| E|CS|TAMS 0x00000004e5e00000, 0x00000004e5e00000| Complete -| 136|0x00000004e6000000, 0x00000004e6200000, 0x00000004e6200000|100%| E|CS|TAMS 0x00000004e6000000, 0x00000004e6000000| Complete -| 137|0x00000004e6200000, 0x00000004e6400000, 0x00000004e6400000|100%| E|CS|TAMS 0x00000004e6200000, 0x00000004e6200000| Complete -| 138|0x00000004e6400000, 0x00000004e6600000, 0x00000004e6600000|100%| E|CS|TAMS 0x00000004e6400000, 0x00000004e6400000| Complete -| 139|0x00000004e6600000, 0x00000004e6800000, 0x00000004e6800000|100%| E|CS|TAMS 0x00000004e6600000, 0x00000004e6600000| Complete -| 140|0x00000004e6800000, 0x00000004e6a00000, 0x00000004e6a00000|100%| E|CS|TAMS 0x00000004e6800000, 0x00000004e6800000| Complete -| 141|0x00000004e6a00000, 0x00000004e6c00000, 0x00000004e6c00000|100%| E|CS|TAMS 0x00000004e6a00000, 0x00000004e6a00000| Complete -| 142|0x00000004e6c00000, 0x00000004e6e00000, 0x00000004e6e00000|100%| E|CS|TAMS 0x00000004e6c00000, 0x00000004e6c00000| Complete -| 143|0x00000004e6e00000, 0x00000004e7000000, 0x00000004e7000000|100%| E|CS|TAMS 0x00000004e6e00000, 0x00000004e6e00000| Complete -| 144|0x00000004e7000000, 0x00000004e7200000, 0x00000004e7200000|100%| E|CS|TAMS 0x00000004e7000000, 0x00000004e7000000| Complete -| 145|0x00000004e7200000, 0x00000004e7400000, 0x00000004e7400000|100%| E|CS|TAMS 0x00000004e7200000, 0x00000004e7200000| Complete -| 146|0x00000004e7400000, 0x00000004e7600000, 0x00000004e7600000|100%| E|CS|TAMS 0x00000004e7400000, 0x00000004e7400000| Complete -| 147|0x00000004e7600000, 0x00000004e7800000, 0x00000004e7800000|100%| E|CS|TAMS 0x00000004e7600000, 0x00000004e7600000| Complete -| 148|0x00000004e7800000, 0x00000004e7a00000, 0x00000004e7a00000|100%| E|CS|TAMS 0x00000004e7800000, 0x00000004e7800000| Complete -| 149|0x00000004e7a00000, 0x00000004e7c00000, 0x00000004e7c00000|100%| E|CS|TAMS 0x00000004e7a00000, 0x00000004e7a00000| Complete -| 150|0x00000004e7c00000, 0x00000004e7e00000, 0x00000004e7e00000|100%| E|CS|TAMS 0x00000004e7c00000, 0x00000004e7c00000| Complete -| 151|0x00000004e7e00000, 0x00000004e8000000, 0x00000004e8000000|100%| E|CS|TAMS 0x00000004e7e00000, 0x00000004e7e00000| Complete -| 152|0x00000004e8000000, 0x00000004e8200000, 0x00000004e8200000|100%| E|CS|TAMS 0x00000004e8000000, 0x00000004e8000000| Complete -| 153|0x00000004e8200000, 0x00000004e8400000, 0x00000004e8400000|100%| E|CS|TAMS 0x00000004e8200000, 0x00000004e8200000| Complete -| 154|0x00000004e8400000, 0x00000004e8600000, 0x00000004e8600000|100%| E|CS|TAMS 0x00000004e8400000, 0x00000004e8400000| Complete -| 155|0x00000004e8600000, 0x00000004e8800000, 0x00000004e8800000|100%| E|CS|TAMS 0x00000004e8600000, 0x00000004e8600000| Complete -| 156|0x00000004e8800000, 0x00000004e8a00000, 0x00000004e8a00000|100%| E|CS|TAMS 0x00000004e8800000, 0x00000004e8800000| Complete -| 157|0x00000004e8a00000, 0x00000004e8c00000, 0x00000004e8c00000|100%| E|CS|TAMS 0x00000004e8a00000, 0x00000004e8a00000| Complete -| 158|0x00000004e8c00000, 0x00000004e8e00000, 0x00000004e8e00000|100%| E|CS|TAMS 0x00000004e8c00000, 0x00000004e8c00000| Complete -| 339|0x00000004ff600000, 0x00000004ff800000, 0x00000004ff800000|100%| E|CS|TAMS 0x00000004ff600000, 0x00000004ff600000| Complete -| 340|0x00000004ff800000, 0x00000004ffa00000, 0x00000004ffa00000|100%| E|CS|TAMS 0x00000004ff800000, 0x00000004ff800000| Complete -| 440|0x000000050c000000, 0x000000050c200000, 0x000000050c200000|100%| E|CS|TAMS 0x000000050c000000, 0x000000050c000000| Complete -| 441|0x000000050c200000, 0x000000050c400000, 0x000000050c400000|100%| E|CS|TAMS 0x000000050c200000, 0x000000050c200000| Complete -| 442|0x000000050c400000, 0x000000050c600000, 0x000000050c600000|100%| E|CS|TAMS 0x000000050c400000, 0x000000050c400000| Complete -| 767|0x0000000534e00000, 0x0000000535000000, 0x0000000535000000|100%| E|CS|TAMS 0x0000000534e00000, 0x0000000534e00000| Complete -|2046|0x00000005d4c00000, 0x00000005d4d74000, 0x00000005d4e00000| 72%|OA| |TAMS 0x00000005d4d74000, 0x00000005d4c00000| Untracked -|2047|0x00000005d4e00000, 0x00000005d4e74000, 0x00000005d5000000| 22%|CA| |TAMS 0x00000005d4e74000, 0x00000005d4e00000| Untracked - -Card table byte_map: [0x0000000107774000,0x0000000107f74000] _byte_map_base: 0x00000001050cc000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x0000000102b9e8c0, (CMBitMap*) 0x0000000102b9e900 - Prev Bits: [0x00000001112d0000, 0x00000001152d0000) - Next Bits: [0x00000001152d0000, 0x00000001192d0000) - -Polling page: 0x0000000102340000 - -Metaspace: - -Usage: - Non-class: 130.13 MB used. - Class: 17.80 MB used. - Both: 147.93 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 130.81 MB ( 68%) committed, 3 nodes. - Class space: 1.00 GB reserved, 18.38 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 149.19 MB ( 12%) committed. - -Chunk freelists: - Non-Class: 12.98 MB - Class: 13.67 MB - Both: 26.65 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 164.69 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 12. -num_arena_births: 2070. -num_arena_deaths: 0. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2384. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 12. -num_chunks_taken_from_freelist: 7802. -num_chunk_merges: 12. -num_chunk_splits: 5805. -num_chunks_enlarged: 4373. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=9105Kb max_used=9105Kb free=110062Kb - bounds [0x00000001241a4000, 0x0000000124a94000, 0x000000012b604000] -CodeHeap 'profiled nmethods': size=119152Kb used=20927Kb max_used=20927Kb free=98225Kb - bounds [0x000000011c604000, 0x000000011da74000, 0x0000000123a60000] -CodeHeap 'non-nmethods': size=7440Kb used=3322Kb max_used=3446Kb free=4117Kb - bounds [0x0000000123a60000, 0x0000000123dd0000, 0x00000001241a4000] - total_blobs=12431 nmethods=11369 adapters=973 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 21.548 Thread 0x000000093dbbb000 nmethod 14855 0x000000011da66990 code [0x000000011da66b80, 0x000000011da66e28] -Event: 21.548 Thread 0x000000093dbbb000 14853 3 org.apache.comet.serde.Config$ConfigMap$Builder::internalGetMutableEntries (110 bytes) -Event: 21.548 Thread 0x000000093dbbaa00 nmethod 14857 0x000000011da66f90 code [0x000000011da67180, 0x000000011da67438] -Event: 21.548 Thread 0x000000093dbbaa00 14860 3 org.apache.spark.sql.internal.SQLConf$$Lambda$514/0x000000e0013364f8:: (20 bytes) -Event: 21.548 Thread 0x000000093dbbaa00 nmethod 14860 0x000000011da67610 code [0x000000011da677c0, 0x000000011da67a18] -Event: 21.548 Thread 0x000000093dbbaa00 14861 3 org.apache.spark.sql.internal.SQLConf::getConfString (50 bytes) -Event: 21.548 Thread 0x000000093dbbb600 nmethod 14850 0x000000011da67b10 code [0x000000011da67d40, 0x000000011da682c8] -Event: 21.548 Thread 0x000000093dbbb600 14862 3 scala.collection.immutable.List::apply (15 bytes) -Event: 21.548 Thread 0x000000093dbbb000 nmethod 14853 0x000000011da68590 code [0x000000011da687c0, 0x000000011da68d58] -Event: 21.548 Thread 0x000000093dbbb000 14863 3 org.apache.logging.log4j.core.pattern.MessagePatternConverter$SimpleMessagePatternConverter::format (87 bytes) -Event: 21.548 Thread 0x000000093dbbb600 nmethod 14862 0x000000011da68f10 code [0x000000011da690c0, 0x000000011da692b8] -Event: 21.548 Thread 0x000000093e8d8000 14864 4 org.apache.spark.internal.config.ConfigReader::$jacocoInit (53 bytes) -Event: 21.548 Thread 0x000000093dbbb600 14865 3 org.apache.comet.ConfigEntry::key (13 bytes) -Event: 21.548 Thread 0x000000093dbbb600 nmethod 14865 0x000000011da69410 code [0x000000011da695c0, 0x000000011da69748] -Event: 21.548 Thread 0x000000093dbbb000 nmethod 14863 0x000000011da69890 code [0x000000011da69b00, 0x000000011da6a408] -Event: 21.549 Thread 0x000000093e8d8000 nmethod 14864 0x0000000124a6d790 code [0x0000000124a6d940, 0x0000000124a6d9e0] -Event: 21.549 Thread 0x000000093e8d8000 14866 4 scala.collection.mutable.HashTable$::nextPositivePowerOfTwo (19 bytes) -Event: 21.549 Thread 0x000000093dbbaa00 nmethod 14861 0x000000011da6a710 code [0x000000011da6aa00, 0x000000011da6b538] -Event: 21.549 Thread 0x000000093e872a00 nmethod 14854 0x000000011da6b990 code [0x000000011da6c040, 0x000000011da6f468] -Event: 21.549 Thread 0x000000093e8d8000 nmethod 14866 0x0000000124a6da90 code [0x0000000124a6dc40, 0x0000000124a6dd30] - -GC Heap History (20 events): -Event: 6.529 GC heap before -{Heap before GC invocations=25 (full 0): - garbage-first heap total 342016K, used 286067K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 87 young (178176K), 13 survivors (26624K) - Metaspace used 123550K, committed 124416K, reserved 1179648K - class space used 14868K, committed 15296K, reserved 1048576K -} -Event: 6.532 GC heap after -{Heap after GC invocations=26 (full 0): - garbage-first heap total 342016K, used 147927K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 6 young (12288K), 6 survivors (12288K) - Metaspace used 123550K, committed 124416K, reserved 1179648K - class space used 14868K, committed 15296K, reserved 1048576K -} -Event: 6.821 GC heap before -{Heap before GC invocations=26 (full 0): - garbage-first heap total 342016K, used 276951K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 70 young (143360K), 6 survivors (12288K) - Metaspace used 127470K, committed 128384K, reserved 1179648K - class space used 15125K, committed 15552K, reserved 1048576K -} -Event: 6.823 GC heap after -{Heap after GC invocations=27 (full 0): - garbage-first heap total 342016K, used 147833K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 6 young (12288K), 6 survivors (12288K) - Metaspace used 127470K, committed 128384K, reserved 1179648K - class space used 15125K, committed 15552K, reserved 1048576K -} -Event: 6.992 GC heap before -{Heap before GC invocations=27 (full 0): - garbage-first heap total 342016K, used 291193K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 71 young (145408K), 6 survivors (12288K) - Metaspace used 133222K, committed 134144K, reserved 1179648K - class space used 15928K, committed 16384K, reserved 1048576K -} -Event: 6.993 GC heap after -{Heap after GC invocations=28 (full 0): - garbage-first heap total 342016K, used 134137K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 133222K, committed 134144K, reserved 1179648K - class space used 15928K, committed 16384K, reserved 1048576K -} -Event: 7.258 GC heap before -{Heap before GC invocations=28 (full 0): - garbage-first heap total 342016K, used 289785K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 74 young (151552K), 2 survivors (4096K) - Metaspace used 139905K, committed 140864K, reserved 1179648K - class space used 16753K, committed 17216K, reserved 1048576K -} -Event: 7.259 GC heap after -{Heap after GC invocations=29 (full 0): - garbage-first heap total 342016K, used 135614K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 139905K, committed 140864K, reserved 1179648K - class space used 16753K, committed 17216K, reserved 1048576K -} -Event: 7.470 GC heap before -{Heap before GC invocations=29 (full 0): - garbage-first heap total 342016K, used 289214K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 76 young (155648K), 3 survivors (6144K) - Metaspace used 147625K, committed 148736K, reserved 1179648K - class space used 17607K, committed 18112K, reserved 1048576K -} -Event: 7.472 GC heap after -{Heap after GC invocations=30 (full 0): - garbage-first heap total 342016K, used 137564K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 147625K, committed 148736K, reserved 1179648K - class space used 17607K, committed 18112K, reserved 1048576K -} -Event: 7.851 GC heap before -{Heap before GC invocations=30 (full 0): - garbage-first heap total 342016K, used 285020K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 76 young (155648K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 7.853 GC heap after -{Heap after GC invocations=31 (full 0): - garbage-first heap total 342016K, used 137850K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.168 GC heap before -{Heap before GC invocations=31 (full 0): - garbage-first heap total 342016K, used 289402K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 78 young (159744K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.170 GC heap after -{Heap after GC invocations=32 (full 0): - garbage-first heap total 342016K, used 137747K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.497 GC heap before -{Heap before GC invocations=32 (full 0): - garbage-first heap total 342016K, used 291347K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 79 young (161792K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.498 GC heap after -{Heap after GC invocations=33 (full 0): - garbage-first heap total 342016K, used 137753K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.814 GC heap before -{Heap before GC invocations=33 (full 0): - garbage-first heap total 342016K, used 293401K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 80 young (163840K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 8.815 GC heap after -{Heap after GC invocations=34 (full 0): - garbage-first heap total 342016K, used 137757K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 9.136 GC heap before -{Heap before GC invocations=34 (full 0): - garbage-first heap total 342016K, used 293405K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 80 young (163840K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} -Event: 9.138 GC heap after -{Heap after GC invocations=35 (full 0): - garbage-first heap total 342016K, used 137909K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 148901K, committed 150016K, reserved 1245184K - class space used 17797K, committed 18304K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.418 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li -Event: 3.418 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.418 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.403 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-6348563280641709946.dylib -Event: 5.871 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 5.871 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 5.871 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 5.871 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 5.871 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 5.871 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 5.871 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.871 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 5.871 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 5.872 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 5.872 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 5.872 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 5.872 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 6.011 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-1210006264174898416.dylib -Event: 6.749 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-5955df27-d344-4639-975a-b5e107a03f33-libsnappyjava.dylib - -Deoptimization events (20 events): -Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: trap_request=0xffffffde fr.pc=0x0000000124a380fc relative=0x0000000000000fbc -Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: reason=class_check action=maybe_recompile pc=0x0000000124a380fc method=scala.collection.TraversableOnce.sum(Lscala/math/Numeric;)Ljava/lang/Object; @ 6 c2 -Event: 21.539 Thread 0x000000093d9d8600 DEOPT PACKING pc=0x0000000124a380fc sp=0x000000017ea89090 -Event: 21.539 Thread 0x000000093d9d8600 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ea88f90 mode 2 -Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c -Event: 21.539 Thread 0x000000093d9d8600 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 21.539 Thread 0x000000093d9d8600 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017ea88ef0 -Event: 21.539 Thread 0x000000093d9d8600 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ea88df0 mode 2 -Event: 21.539 Thread 0x000000093d9d8000 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c -Event: 21.539 Thread 0x000000093d9d8000 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 21.539 Thread 0x000000093d9d9200 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b0c3c relative=0x000000000000013c -Event: 21.539 Thread 0x000000093d9d8000 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017e67cf50 -Event: 21.539 Thread 0x000000093d9d9200 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b0c3c method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 21.539 Thread 0x000000093d9d9200 DEOPT PACKING pc=0x00000001249b0c3c sp=0x000000017f2a0f50 -Event: 21.539 Thread 0x000000093d9d8000 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017e67ce50 mode 2 -Event: 21.539 Thread 0x000000093d9d9200 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017f2a0e50 mode 2 -Event: 21.539 Thread 0x000000093d9d8c00 Uncommon trap: trap_request=0xffffffde fr.pc=0x00000001249b06a8 relative=0x00000000000000e8 -Event: 21.539 Thread 0x000000093d9d8c00 Uncommon trap: reason=class_check action=maybe_recompile pc=0x00000001249b06a8 method=scala.math.Numeric.zero()Ljava/lang/Object; @ 6 c2 -Event: 21.539 Thread 0x000000093d9d8c00 DEOPT PACKING pc=0x00000001249b06a8 sp=0x000000017ee94ed0 -Event: 21.539 Thread 0x000000093d9d8c00 DEOPT UNPACKING pc=0x0000000123aa791c sp=0x000000017ee94e50 mode 2 - -Classes loaded (20 events): -Event: 7.438 Loading class java/lang/Class$EnclosingMethodInfo -Event: 7.438 Loading class java/lang/Class$EnclosingMethodInfo done -Event: 15.240 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper -Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$Unmapper -Event: 15.241 Loading class jdk/internal/access/foreign/UnmapperProxy -Event: 15.241 Loading class jdk/internal/access/foreign/UnmapperProxy done -Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$Unmapper done -Event: 15.241 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done -Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask -Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask -Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done -Event: 20.226 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done -Event: 20.226 Loading class java/util/function/LongBinaryOperator -Event: 20.226 Loading class java/util/function/LongBinaryOperator done -Event: 21.429 Loading class sun/nio/ch/ChannelInputStream -Event: 21.429 Loading class sun/nio/ch/ChannelInputStream done -Event: 21.482 Loading class java/math/BigDecimal$StringBuilderHelper -Event: 21.482 Loading class java/math/BigDecimal$StringBuilderHelper done -Event: 21.544 Loading class java/nio/channels/Channels$ReadableByteChannelImpl -Event: 21.544 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 21.540 Thread 0x000000093d9d9200 Exception (0x00000004e33ed9f0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.540 Thread 0x000000093d9d8c00 Exception (0x00000004e388ac00) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.540 Thread 0x000000093d9d8000 Exception (0x00000004e35ba828) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.540 Thread 0x000000093d9d8600 Exception (0x00000004e33737a8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.540 Thread 0x000000093d9bf600 Exception (0x00000004e32de4e0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e30134a0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e241b0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e3384178) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32f2ca8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3152498) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e28030) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e3387f50) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e3017478) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3156270) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32f6a80) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d9200 Exception (0x00000004e2e2b910) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8600 Exception (0x00000004e338b7e0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8c00 Exception (0x00000004e301ad08) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9d8000 Exception (0x00000004e3159b00) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.542 Thread 0x000000093d9bf600 Exception (0x00000004e32fa310) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] - -VM Operations (20 events): -Event: 8.168 Executing VM operation: G1CollectForAllocation -Event: 8.170 Executing VM operation: G1CollectForAllocation done -Event: 8.497 Executing VM operation: G1CollectForAllocation -Event: 8.498 Executing VM operation: G1CollectForAllocation done -Event: 8.814 Executing VM operation: G1CollectForAllocation -Event: 8.815 Executing VM operation: G1CollectForAllocation done -Event: 9.136 Executing VM operation: G1CollectForAllocation -Event: 9.138 Executing VM operation: G1CollectForAllocation done -Event: 12.153 Executing VM operation: Cleanup -Event: 12.153 Executing VM operation: Cleanup done -Event: 16.173 Executing VM operation: Cleanup -Event: 16.173 Executing VM operation: Cleanup done -Event: 21.198 Executing VM operation: Cleanup -Event: 21.198 Executing VM operation: Cleanup done -Event: 21.472 Executing VM operation: ICBufferFull -Event: 21.472 Executing VM operation: ICBufferFull done -Event: 21.508 Executing VM operation: ICBufferFull -Event: 21.508 Executing VM operation: ICBufferFull done -Event: 21.548 Executing VM operation: ICBufferFull -Event: 21.548 Executing VM operation: ICBufferFull done - -Memory protections (20 events): -Event: 7.060 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 3 -Event: 7.060 Protecting memory [0x0000000300410000,0x000000030041c000] with protection modes 3 -Event: 7.266 Protecting memory [0x0000000300410000,0x000000030041c000] with protection modes 0 -Event: 7.269 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 0 -Event: 7.269 Protecting memory [0x0000000300c28000,0x0000000300c34000] with protection modes 0 -Event: 7.270 Protecting memory [0x0000000301034000,0x0000000301040000] with protection modes 0 -Event: 7.270 Protecting memory [0x0000000301440000,0x000000030144c000] with protection modes 0 -Event: 9.274 Protecting memory [0x0000000171164000,0x0000000171170000] with protection modes 3 -Event: 14.280 Protecting memory [0x00000001748d0000,0x00000001748dc000] with protection modes 3 -Event: 15.240 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 3 -Event: 20.228 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 0 -Event: 21.428 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 3 -Event: 21.428 Protecting memory [0x0000000170d4c000,0x0000000170d58000] with protection modes 3 -Event: 21.432 Protecting memory [0x0000000170d4c000,0x0000000170d58000] with protection modes 0 -Event: 21.447 Protecting memory [0x0000000170f58000,0x0000000170f64000] with protection modes 0 -Event: 21.447 Protecting memory [0x0000000171164000,0x0000000171170000] with protection modes 0 -Event: 21.447 Protecting memory [0x00000001748d0000,0x00000001748dc000] with protection modes 0 -Event: 21.460 Protecting memory [0x0000000302288000,0x0000000302294000] with protection modes 0 -Event: 21.472 Protecting memory [0x00000003026a0000,0x00000003026ac000] with protection modes 0 -Event: 21.527 Protecting memory [0x0000000302aac000,0x0000000302ab8000] with protection modes 0 - -Nmethod flushes (20 events): -Event: 7.445 Thread 0x000000093e873000 flushing nmethod 0x0000000124854590 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5dfd90 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f2410 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f2890 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f4190 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f5010 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f5e90 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f9310 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5f9890 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5fae90 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d5fee90 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d601690 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d603790 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d604810 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d605690 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d60a210 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d60e910 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d610610 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d613490 -Event: 7.449 Thread 0x000000093e873000 flushing nmethod 0x000000011d61fd10 - -Events (20 events): -Event: 7.060 Thread 0x000000093da19800 Thread exited: 0x000000093da19800 -Event: 7.060 Thread 0x000000093da19200 Thread exited: 0x000000093da19200 -Event: 7.266 Thread 0x000000093e752400 Thread added: 0x000000093db16400 -Event: 7.269 Thread 0x000000093db16400 Thread added: 0x000000093db17000 -Event: 7.269 Thread 0x000000093db16400 Thread added: 0x000000093db17600 -Event: 7.270 Thread 0x000000093db17600 Thread added: 0x000000093db68000 -Event: 7.270 Thread 0x000000093db68000 Thread added: 0x000000093db68600 -Event: 9.274 Thread 0x000000093d9bd800 Thread exited: 0x000000093d9bd800 -Event: 14.280 Thread 0x000000093d9d9800 Thread exited: 0x000000093d9d9800 -Event: 15.240 Thread 0x000000093e8d8600 Thread exited: 0x000000093e8d8600 -Event: 20.228 Thread 0x000000093e872a00 Thread added: 0x000000093dbba400 -Event: 21.428 Thread 0x000000093dbba400 Thread exited: 0x000000093dbba400 -Event: 21.428 Thread 0x000000093e8d8000 Thread exited: 0x000000093e8d8000 -Event: 21.432 Thread 0x000000093e872a00 Thread added: 0x000000093dbba400 -Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbaa00 -Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbb000 -Event: 21.447 Thread 0x000000093e872a00 Thread added: 0x000000093dbbb600 -Event: 21.460 Thread 0x000000093dbbb600 Thread added: 0x000000093e8d8000 -Event: 21.472 Thread 0x000000093e785e00 Thread added: 0x000000093db6b600 -Event: 21.527 Thread 0x000000093e872400 Thread added: 0x000000093b2acc00 - - -Dynamic libraries: -0x0000000102358000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x0000000107f8c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x0000000102380000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x00000001023b0000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x0000000102400000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x000000011b3f8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x000000010266c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x000000011b444000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x000000011b420000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x000000011b4e8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x000000011b4fc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x000000011b510000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x000000011b524000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000147dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-6348563280641709946.dylib -0x000000011b564000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-1210006264174898416.dylib -0x000000011b540000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-5955df27-d344-4639-975a-b5e107a03f33-libsnappyjava.dylib - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:41 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 8.44 7.59 5.66 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(3784624k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. diff --git a/spark/hs_err_pid59395.log b/spark/hs_err_pid59395.log deleted file mode 100644 index 32c9dedeb3..0000000000 --- a/spark/hs_err_pid59395.log +++ /dev/null @@ -1,1886 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x0000000108215530, pid=59395, tid=66563 -# -# JRE version: OpenJDK Runtime Environment Temurin-17.0.17+10 (17.0.17+10) (build 17.0.17+10) -# Java VM: OpenJDK 64-Bit Server VM Temurin-17.0.17+10 (17.0.17+10, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, bsd-aarch64) -# Problematic frame: -# V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -# -# No core dump will be written. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/adoptium/adoptium-support/issues -# - ---------------- S U M M A R Y ------------ - -Command Line: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. - -Host: "Mac15,14" arm64, 28 cores, 96G, Darwin 25.3.0, macOS 26.3.1 (25D2128) -Time: Sun Apr 12 09:00:35 2026 MDT elapsed time: 21.311638 seconds (0d 0h 0m 21s) - ---------------- T H R E A D --------------- - -Current thread (0x0000000afb041e00): JavaThread "Executor task launch worker for task 29.0 in stage 4.0 (TID 40)" daemon [_thread_in_vm, id=66563, stack(0x000000017ec70000,0x000000017f073000)] - -Stack: [0x000000017ec70000,0x000000017f073000], sp=0x000000017f06ac00, free space=4075k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x4bd530] jni_invoke_nonstatic(JNIEnv_*, JavaValue*, _jobject*, JNICallType, _jmethodID*, JNI_ArgumentPusher*, JavaThread*)+0x210 -V [libjvm.dylib+0x4c160c] jni_CallIntMethodA+0x138 -C [libcomet-766714755951389016.dylib+0x79560] jni::env::Env::call_method_unchecked::h99ac6bddd8d1ce0e+0xdc8 -C [libcomet-766714755951389016.dylib+0x260a0c] comet::execution::operators::scan::ScanExec::get_next::_$u7b$$u7b$closure$u7d$$u7d$::hf024c8328cafad1e+0xbc -C [libcomet-766714755951389016.dylib+0x19050c] datafusion_comet_jni_bridge::JVMClasses::with_env::h3122499bda05b0fb+0x208 -C [libcomet-766714755951389016.dylib+0x14712c] comet::execution::operators::scan::ScanExec::get_next::h8982ff7ce2e8c294+0xa4 -C [libcomet-766714755951389016.dylib+0x144ddc] comet::execution::operators::scan::ScanExec::get_next_batch::h7c9f45245d6ef67c+0x29c -C [libcomet-766714755951389016.dylib+0x165c0] comet::execution::jni_api::pull_input_batches::_$u7b$$u7b$closure$u7d$$u7d$::hfc68478584806621+0x34 -C [libcomet-766714755951389016.dylib+0x1f97c0] core::iter::traits::iterator::Iterator::try_for_each::call::_$u7b$$u7b$closure$u7d$$u7d$::hed1de203fbe31f38+0x20 -C [libcomet-766714755951389016.dylib+0x1722fc] core::iter::traits::iterator::Iterator::try_fold::h0e5db3f49aa4887d+0xc4 -C [libcomet-766714755951389016.dylib+0x170730] core::iter::traits::iterator::Iterator::try_for_each::h92b6bb209cc06cc8+0x18 -C [libcomet-766714755951389016.dylib+0x1d77fc] comet::execution::jni_api::pull_input_batches::h54403f6453112ca5+0x44 -C [libcomet-766714755951389016.dylib+0x1ab70] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h7c7fa1f879d37191+0x18 -C [libcomet-766714755951389016.dylib+0x1129dc] tokio::runtime::context::runtime_mt::exit_runtime::h35beabe339a354e7+0x70 -C [libcomet-766714755951389016.dylib+0x9cf84] tokio::runtime::scheduler::multi_thread::worker::block_in_place::hacf6b12bc86e8e33+0x12c -C [libcomet-766714755951389016.dylib+0x19ddcc] tokio::runtime::scheduler::block_in_place::block_in_place::hbc3c977b4cdec3c3+0x18 -C [libcomet-766714755951389016.dylib+0x1d9794] tokio::task::blocking::block_in_place::he3618eb1e7cc0c3b+0x18 -C [libcomet-766714755951389016.dylib+0x1a46c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::hcc95bd828abf3c5b+0x41c -C [libcomet-766714755951389016.dylib+0xbf0b0] tokio::runtime::park::CachedParkThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::h05f17cce0e9bc326+0x40 -C [libcomet-766714755951389016.dylib+0xbb1ac] tokio::runtime::park::CachedParkThread::block_on::h119919521b569743+0x238 -C [libcomet-766714755951389016.dylib+0x326104] tokio::runtime::context::blocking::BlockingRegionGuard::block_on::h2e9e5282eccf6ad3+0x74 -C [libcomet-766714755951389016.dylib+0x170200] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::_$u7b$$u7b$closure$u7d$$u7d$::hb06e5d72589e4f3c+0x4c -C [libcomet-766714755951389016.dylib+0x18c0ec] tokio::runtime::context::runtime::enter_runtime::h8a7834613f984e72+0xe8 -C [libcomet-766714755951389016.dylib+0x16ff18] tokio::runtime::scheduler::multi_thread::MultiThread::block_on::hc8b348caf3aa3417+0x58 -C [libcomet-766714755951389016.dylib+0x19247c] tokio::runtime::runtime::Runtime::block_on_inner::hc94c8f06d2c9555e+0xb4 -C [libcomet-766714755951389016.dylib+0x192e70] tokio::runtime::runtime::Runtime::block_on::h4ae6ded1779d3acb+0x15c -C [libcomet-766714755951389016.dylib+0x197a0] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::_$u7b$$u7b$closure$u7d$$u7d$::h61ffeb9485efac5d+0xa1c -C [libcomet-766714755951389016.dylib+0x1c9498] datafusion_comet_common::tracing::with_trace::h642a97017f156be8+0x6c -C [libcomet-766714755951389016.dylib+0x18c5c] comet::execution::jni_api::Java_org_apache_comet_Native_executePlan::_$u7b$$u7b$closure$u7d$$u7d$::h6996b333f118bf6d+0xcc -C [libcomet-766714755951389016.dylib+0x740cc] jni::env::EnvUnowned::with_env::_$u7b$$u7b$closure$u7d$$u7d$::h638dee1279224d79+0x40 -C [libcomet-766714755951389016.dylib+0x13a32c] _$LT$core..panic..unwind_safe..AssertUnwindSafe$LT$F$GT$$u20$as$u20$core..ops..function..FnOnce$LT$$LP$$RP$$GT$$GT$::call_once::hab419cb48d0afbf9+0x34 -C [libcomet-766714755951389016.dylib+0x2efb64] std::panicking::catch_unwind::do_call::h1c045baaa8f6cfc5+0x44 -C [libcomet-766714755951389016.dylib+0x106628] __rust_try+0x20 -C [libcomet-766714755951389016.dylib+0x104ee0] std::panic::catch_unwind::hab0e396380744b21+0x78 -C [libcomet-766714755951389016.dylib+0x70d00] jni::env::EnvUnowned::with_env::h12e3d990b3f6ba00+0xd8 -C [libcomet-766714755951389016.dylib+0x35bedc] datafusion_comet_jni_bridge::errors::try_unwrap_or_throw::h5fa3bd78b34be54a+0xac -C [libcomet-766714755951389016.dylib+0x1d5e6c] Java_org_apache_comet_Native_executePlan+0x60 -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3957+0x000000d002198000.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3950+0x000000d0021966b0.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2515+0x000000d001df15b8.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub -V [libjvm.dylib+0x45e454] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x3d8 -V [libjvm.dylib+0x45d3d4] JavaCalls::call_virtual(JavaValue*, Klass*, Symbol*, Symbol*, JavaCallArguments*, JavaThread*)+0x11c -V [libjvm.dylib+0x45d4a0] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x64 -V [libjvm.dylib+0x515424] thread_entry(JavaThread*, JavaThread*)+0xc4 -V [libjvm.dylib+0x9a786c] JavaThread::thread_main_inner()+0x148 -V [libjvm.dylib+0x9a5f30] Thread::call_run()+0xcc -V [libjvm.dylib+0x7abca4] thread_native_entry(Thread*)+0x158 -C [libsystem_pthread.dylib+0x6c08] _pthread_start+0x88 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j org.apache.comet.Native.executePlan(IIJ[J[J)J+0 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)J+23 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$2$adapted(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;[J[J)Ljava/lang/Object;+9 -j org.apache.comet.CometExecIterator$$Lambda$3957+0x000000d002198000.apply(Ljava/lang/Object;Ljava/lang/Object;)Ljava/lang/Object;+16 -j org.apache.comet.vector.NativeUtil.getNextBatch(ILscala/Function2;)Lscala/Option;+201 -j org.apache.comet.CometExecIterator.$anonfun$getNextBatch$1(Lorg/apache/comet/CometExecIterator;Lorg/apache/spark/TaskContext;)Lscala/Option;+25 -j org.apache.comet.CometExecIterator$$Lambda$3950+0x000000d0021966b0.apply()Ljava/lang/Object;+8 -j org.apache.comet.Tracing$.withTrace(Ljava/lang/String;ZLscala/Function0;)Ljava/lang/Object;+39 -j org.apache.comet.CometExecIterator.getNextBatch()Lscala/Option;+144 -j org.apache.comet.CometExecIterator.hasNext()Z+197 -j org.apache.spark.sql.comet.execution.shuffle.CometNativeShuffleWriter.write(Lscala/collection/Iterator;)V+747 -j org.apache.spark.shuffle.ShuffleWriteProcessor.write(Lorg/apache/spark/rdd/RDD;Lorg/apache/spark/ShuffleDependency;JLorg/apache/spark/TaskContext;Lorg/apache/spark/Partition;)Lorg/apache/spark/scheduler/MapStatus;+75 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Lorg/apache/spark/scheduler/MapStatus;+300 -j org.apache.spark.scheduler.ShuffleMapTask.runTask(Lorg/apache/spark/TaskContext;)Ljava/lang/Object;+6 -j org.apache.spark.TaskContext.runTaskWithListeners(Lorg/apache/spark/scheduler/Task;)Ljava/lang/Object;+6 -j org.apache.spark.scheduler.Task.run(JILorg/apache/spark/metrics/MetricsSystem;ILscala/collection/immutable/Map;Lscala/Option;)Ljava/lang/Object;+454 -j org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Lorg/apache/spark/executor/Executor$TaskRunner;Lscala/runtime/BooleanRef;)Ljava/lang/Object;+83 -j org.apache.spark.executor.Executor$TaskRunner$$Lambda$2515+0x000000d001df15b8.apply()Ljava/lang/Object;+8 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+15 -j org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(Lorg/apache/spark/util/SparkErrorUtils;Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.util.Utils$.tryWithSafeFinally(Lscala/Function0;Lscala/Function0;)Ljava/lang/Object;+8 -j org.apache.spark.executor.Executor$TaskRunner.run()V+797 -j java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V+92 java.base@17.0.17 -j java.util.concurrent.ThreadPoolExecutor$Worker.run()V+5 java.base@17.0.17 -j java.lang.Thread.run()V+11 java.base@17.0.17 -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 2 (SEGV_ACCERR), si_addr: 0x0000000000000029 - -Registers: - x0=0x0000000000000000 x1=0x0000000000000007 x2=0x0000000000000001 x3=0x0000000b069bc1f0 - x4=0x000000017f06ad20 x5=0x0000000afb041e00 x6=0x0000000b025d7b00 x7=0xfffff0003ffff800 - x8=0x000000d00217f978 x9=0x0000000000000000 x10=0x000000d000000000 x11=0x0000000000000004 -x12=0x0000000137020e48 x13=0x0000000000000000 x14=0xfffffffffffdb114 x15=0x0000000b025cc000 -x16=0x0000000000000040 x17=0x0000000b02016290 x18=0x0000000000000000 x19=0x0000000afb041e00 -x20=0x000000017f06ad50 x21=0x000000017f06ad20 x22=0x0000000afbc68018 x23=0x00000005016d06b0 -x24=0x0000000000000001 x25=0x0000000000000000 x26=0x0000000b02033700 x27=0x0000000000000000 -x28=0x0000000000000001 fp=0x000000017f06ad10 lr=0x000000010821552c sp=0x000000017f06ac00 -pc=0x0000000108215530 cpsr=0x0000000000001000 - -Register to memory mapping: - - x0=0x0 is NULL - x1=0x0000000000000007 is an unknown value - x2=0x0000000000000001 is an unknown value - x3=0x0000000b069bc1f0 points into unknown readable memory: 0x0000000137020e48 | 48 0e 02 37 01 00 00 00 - x4=0x000000017f06ad20 is pointing into the stack for thread: 0x0000000afb041e00 - x5=0x0000000afb041e00 is a thread - x6=0x0000000b025d7b00 points into unknown readable memory: 0x0000000af564c600 | 00 c6 64 f5 0a 00 00 00 - x7=0xfffff0003ffff800 is an unknown value - x8=0x000000d00217f978 is pointing into metadata - x9=0x0 is NULL -x10=0x000000d000000000 is pointing into metadata -x11=0x0000000000000004 is an unknown value -x12={method} {0x0000000137020e48} 'hasNext' '()I' in 'org/apache/comet/CometBatchIterator' -x13=0x0 is NULL -x14=0xfffffffffffdb114 is an unknown value -x15=0x0000000b025cc000 points into unknown readable memory: 0x0000004000000004 | 04 00 00 00 40 00 00 00 -x16=0x0000000000000040 is an unknown value -x17=0x0000000b02016290 points into unknown readable memory: 0xfffffffffffffffe | fe ff ff ff ff ff ff ff -x18=0x0 is NULL -x19=0x0000000afb041e00 is a thread -x20=0x000000017f06ad50 is pointing into the stack for thread: 0x0000000afb041e00 -x21=0x000000017f06ad20 is pointing into the stack for thread: 0x0000000afb041e00 -x22=0x0000000afbc68018 points into unknown readable memory: 0x00000005016d06b0 | b0 06 6d 01 05 00 00 00 -x23=0x00000005016d06b0 is an oop: org.apache.comet.CometHandleBatchIterator -{0x00000005016d06b0} - klass: 'org/apache/comet/CometHandleBatchIterator' - - ---- fields (total size 2 words): - - private final 'source' 'Lorg/apache/comet/CometExecIterator;' @12 a 'org/apache/comet/CometExecIterator'{0x00000005016c6088} (a02d8c11) -x24=0x0000000000000001 is an unknown value -x25=0x0 is NULL -x26=0x0000000b02033700 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -x27=0x0 is NULL -x28=0x0000000000000001 is an unknown value - - -Top of Stack: (sp=0x000000017f06ac00) -0x000000017f06ac00: 000000017f0652c0 0000000000000001 -0x000000017f06ac10: 0000000b0232d7c0 0000000151212428 -0x000000017f06ac20: 0000000afcc29230 0000000af5652d00 -0x000000017f06ac30: 000000017f06ac67 0000000000000000 -0x000000017f06ac40: 000000017f06ac80 000000014e7800d0 -0x000000017f06ac50: 0000000000000081 0000000af6e0e8d8 -0x000000017f06ac60: 0000000b0232d9a0 0000000af6e0e8d8 -0x000000017f06ac70: 0000000af6e0e8d8 00ffffffffffffff -0x000000017f06ac80: 000000017f06ad10 000000014e7aff58 -0x000000017f06ac90: 0000000000000063 000000017f06ad20 -0x000000017f06aca0: 000000017f06ad10 0000000108231f24 -0x000000017f06acb0: ba714bd2f1340072 0000000afb041e00 -0x000000017f06acc0: 0000000afb041e00 0000000000000000 -0x000000017f06acd0: 0000000136fa8ab0 0000000000000000 -0x000000017f06ace0: 0000000afb0420b0 0000000000000008 -0x000000017f06acf0: 0000000af6eb1bf8 0000000b069bc1f0 -0x000000017f06ad00: 0000000000000001 0000000afb041e00 -0x000000017f06ad10: 000000017f06adc0 000000010821960c -0x000000017f06ad20: 000000010895a6b8 000000d00045a1d0 -0x000000017f06ad30: 000000010000000a 0000000000000014 -0x000000017f06ad40: 0000000000000000 0000000000000008 -0x000000017f06ad50: 000000000000000a 00000001089c3f90 -0x000000017f06ad60: 0000000afb041e00 0000000000000000 -0x000000017f06ad70: 000000017f06adc0 0000000147e84c58 -0x000000017f06ad80: 0000000136fa8ab0 00000005016d8a98 -0x000000017f06ad90: 000000017f071818 0000000000000061 -0x000000017f06ada0: 0000000000000000 00000001089c3f90 -0x000000017f06adb0: 000000017f071738 000000012426772c -0x000000017f06adc0: 000000017f06c980 0000000147e49560 -0x000000017f06add0: 0000000000000000 0000000000000081 -0x000000017f06ade0: 0000000000000082 0000000b0232d9a0 -0x000000017f06adf0: 0000000000000082 000000ffffffffff - -Instructions: (pc=0x0000000108215530) -0x0000000108215430: 04 00 80 d2 fd 7b 51 a9 f4 4f 50 a9 f6 57 4f a9 -0x0000000108215440: f8 5f 4e a9 fa 67 4d a9 fc 6f 4c a9 ff 83 04 91 -0x0000000108215450: 95 e0 f9 17 48 0f 00 f9 d7 02 00 f9 39 03 40 f9 -0x0000000108215460: 28 07 40 f9 1c 59 40 79 1f 07 00 71 21 06 00 54 -0x0000000108215470: 21 2f 40 b9 3f 24 00 31 ca 02 00 54 08 05 40 f9 -0x0000000108215480: 08 0d 40 f9 29 01 80 12 22 01 01 4b c9 3c 00 d0 -0x0000000108215490: 29 65 0a 91 29 01 40 39 89 03 00 34 e9 0a 40 b9 -0x00000001082154a0: 2a 3b 00 b0 4a 41 24 91 4b 01 40 f9 4a 09 40 b9 -0x00000001082154b0: 29 21 ca 9a 20 01 0b 8b e1 03 08 aa e3 03 13 aa -0x00000001082154c0: 8f 15 fe 97 68 06 40 f9 a8 12 00 b5 18 00 00 14 -0x00000001082154d0: 3f 08 00 31 e0 02 00 54 c8 3c 00 d0 08 65 0a 91 -0x00000001082154e0: 08 01 40 39 08 02 00 34 e8 0a 40 b9 29 3b 00 b0 -0x00000001082154f0: 29 41 24 91 2a 01 40 f9 29 09 40 b9 08 21 c9 9a -0x0000000108215500: 00 01 0a 8b 09 00 00 14 e0 06 40 f9 e1 03 08 aa -0x0000000108215510: e3 03 13 aa 7a 15 fe 97 68 06 40 f9 08 10 00 b5 -0x0000000108215520: 03 00 00 14 e0 06 40 f9 d6 12 06 94 f9 03 00 aa -0x0000000108215530: 28 a7 40 39 68 04 10 37 f9 4f 02 a9 99 01 00 b4 -0x0000000108215540: 77 26 41 f9 e1 22 40 29 3f 00 08 6b 81 00 00 54 -0x0000000108215550: e0 03 17 aa f3 ff f1 97 e1 02 40 b9 28 04 00 11 -0x0000000108215560: e8 02 00 b9 e8 06 40 f9 19 d9 21 f8 77 1e 41 f9 -0x0000000108215570: f8 6a 41 a9 e8 66 42 a9 e8 0f 00 f9 ff 57 00 f9 -0x0000000108215580: 9f 27 00 71 e3 05 00 54 fa e3 00 a9 9a 07 00 91 -0x0000000108215590: 40 f3 7d d3 01 00 80 52 0a cc 0d 94 fb 03 00 aa -0x00000001082155a0: e0 03 1a aa fa e3 40 a9 01 00 80 52 05 cc 0d 94 -0x00000001082155b0: 68 23 00 91 e8 47 00 f9 08 04 00 91 26 00 00 14 -0x00000001082155c0: 74 1e 41 f9 95 62 41 a9 9a 5a 42 a9 28 3d 00 f0 -0x00000001082155d0: 08 81 1f 91 17 75 42 f9 28 07 40 f9 09 05 40 f9 -0x00000001082155e0: 08 45 40 79 28 0d 08 8b 00 25 40 f9 70 7a 12 94 -0x00000001082155f0: e4 03 00 aa 01 31 00 f0 21 24 2c 91 e0 03 13 aa -0x0000000108215600: 02 75 80 52 e3 03 17 aa 27 e0 f9 97 a8 02 40 f9 -0x0000000108215610: c8 00 00 b4 e0 03 14 aa e1 03 16 aa 29 32 f1 97 -0x0000000108215620: e0 03 15 aa d4 31 f1 97 88 0e 40 f9 1f 01 18 eb - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x000000017f0652c0 is pointing into the stack for thread: 0x0000000afb041e00 -stack at sp + 1 slots: 0x0000000000000001 is an unknown value -stack at sp + 2 slots: 0x0000000b0232d7c0 points into unknown readable memory: 0x0000000000000002 | 02 00 00 00 00 00 00 00 -stack at sp + 3 slots: 0x0000000151212428: _ZN5comet7parquet15parquet_support15PARQUET_OPTIONS17h20af737816b13564E+0x28650 in /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib at 0x0000000147dd0000 -stack at sp + 4 slots: 0x0000000afcc29230 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -stack at sp + 5 slots: 0x0000000af5652d00 points into unknown readable memory: 0x0000000000000001 | 01 00 00 00 00 00 00 00 -stack at sp + 6 slots: 0x000000017f06ac67 is pointing into the stack for thread: 0x0000000afb041e00 -stack at sp + 7 slots: 0x0 is NULL - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x0000000b07053280, length=80, elements={ -0x000000010295df90, 0x0000000102971cf0, 0x0000000afcd80000, 0x0000000afcd80c00, -0x0000000afcd81200, 0x0000000afcd81800, 0x0000000afcd81e00, 0x0000000afcd82400, -0x0000000afcd82a00, 0x0000000afcd83000, 0x0000000afcbcc600, 0x0000000afbceaa00, -0x0000000af9845e00, 0x0000000af9847000, 0x0000000af9847600, 0x0000000afb9c8000, -0x0000000afb9c8600, 0x0000000afb9c8c00, 0x0000000afb9cb000, 0x0000000afb9cb600, -0x0000000af9944000, 0x0000000af9944600, 0x0000000af9944c00, 0x0000000af9945200, -0x0000000af9945800, 0x0000000af9945e00, 0x0000000af9974600, 0x0000000af9976400, -0x0000000af9976a00, 0x0000000af9977600, 0x0000000af9801200, 0x0000000af9ebb000, -0x0000000afcbcde00, 0x0000000afcbcd800, 0x0000000afcd83600, 0x0000000af99d8000, -0x0000000af99d9800, 0x0000000af99d9e00, 0x0000000af99daa00, 0x0000000af99ec000, -0x0000000af99ec600, 0x0000000af99ecc00, 0x0000000af99ed200, 0x0000000af9a89800, -0x0000000af9a8b600, 0x0000000af9ad0600, 0x0000000af9bb8000, 0x0000000af9145800, -0x0000000afb041200, 0x0000000afb041800, 0x0000000afb041e00, 0x0000000afb042400, -0x0000000afb042a00, 0x0000000afb24ea00, 0x0000000af8168000, 0x0000000af8168600, -0x0000000af8169200, 0x0000000afb503600, 0x0000000af816a400, 0x0000000afb5b8000, -0x0000000af943d800, 0x0000000af6862a00, 0x0000000af6863000, 0x0000000af6863600, -0x0000000af99d9200, 0x0000000afbfe4c00, 0x0000000afb24c000, 0x0000000af6861e00, -0x0000000af6e1c000, 0x0000000af6e1c600, 0x0000000af6e1cc00, 0x0000000af68a7600, -0x0000000af8169800, 0x0000000af816aa00, 0x0000000afb470c00, 0x0000000afb471200, -0x0000000af6860c00, 0x0000000af5631e00, 0x0000000af601c600, 0x0000000af601cc00 -} - -Java Threads: ( => current thread ) - 0x000000010295df90 JavaThread "ScalaTest-main-running-CometNativeShuffleSuite" [_thread_blocked, id=4099, stack(0x000000016dd10000,0x000000016e113000)] - 0x0000000102971cf0 JavaThread "Reference Handler" daemon [_thread_blocked, id=32003, stack(0x000000016ed64000,0x000000016f167000)] - 0x0000000afcd80000 JavaThread "Finalizer" daemon [_thread_blocked, id=23811, stack(0x000000016f170000,0x000000016f573000)] - 0x0000000afcd80c00 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=30211, stack(0x000000016f694000,0x000000016fa97000)] - 0x0000000afcd81200 JavaThread "Service Thread" daemon [_thread_blocked, id=26115, stack(0x000000016faa0000,0x000000016fea3000)] - 0x0000000afcd81800 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=29955, stack(0x000000016feac000,0x00000001702af000)] - 0x0000000afcd81e00 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=26883, stack(0x00000001702b8000,0x00000001704bb000)] - 0x0000000afcd82400 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=27139, stack(0x00000001704c4000,0x00000001706c7000)] - 0x0000000afcd82a00 JavaThread "Sweeper thread" daemon [_thread_blocked, id=29443, stack(0x00000001706d0000,0x0000000170ad3000)] - 0x0000000afcd83000 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=29187, stack(0x0000000170adc000,0x0000000170edf000)] - 0x0000000afcbcc600 JavaThread "Notification Thread" daemon [_thread_blocked, id=28163, stack(0x0000000170ee8000,0x00000001712eb000)] - 0x0000000afbceaa00 JavaThread "ScalaTest-dispatcher" [_thread_blocked, id=37635, stack(0x0000000173be4000,0x0000000173fe7000)] - 0x0000000af9845e00 JavaThread "rpc-boss-3-1" daemon [_thread_in_native, id=45827, stack(0x0000000175050000,0x0000000175453000)] - 0x0000000af9847000 JavaThread "dispatcher-event-loop-0" daemon [_thread_blocked, id=46339, stack(0x000000017545c000,0x000000017585f000)] - 0x0000000af9847600 JavaThread "dispatcher-event-loop-1" daemon [_thread_blocked, id=63747, stack(0x0000000175868000,0x0000000175c6b000)] - 0x0000000afb9c8000 JavaThread "dispatcher-event-loop-2" daemon [_thread_blocked, id=46851, stack(0x0000000175c74000,0x0000000176077000)] - 0x0000000afb9c8600 JavaThread "dispatcher-event-loop-3" daemon [_thread_blocked, id=63491, stack(0x0000000176080000,0x0000000176483000)] - 0x0000000afb9c8c00 JavaThread "dispatcher-event-loop-4" daemon [_thread_blocked, id=62979, stack(0x000000017648c000,0x000000017688f000)] - 0x0000000afb9cb000 JavaThread "map-output-dispatcher-0" daemon [_thread_blocked, id=62467, stack(0x0000000176898000,0x0000000176c9b000)] - 0x0000000afb9cb600 JavaThread "map-output-dispatcher-1" daemon [_thread_blocked, id=47363, stack(0x0000000176ca4000,0x00000001770a7000)] - 0x0000000af9944000 JavaThread "map-output-dispatcher-2" daemon [_thread_blocked, id=47875, stack(0x00000001770b0000,0x00000001774b3000)] - 0x0000000af9944600 JavaThread "map-output-dispatcher-3" daemon [_thread_blocked, id=48387, stack(0x00000001774bc000,0x00000001778bf000)] - 0x0000000af9944c00 JavaThread "map-output-dispatcher-4" daemon [_thread_blocked, id=61699, stack(0x00000001778c8000,0x0000000177ccb000)] - 0x0000000af9945200 JavaThread "map-output-dispatcher-5" daemon [_thread_blocked, id=48643, stack(0x0000000177cd4000,0x00000001780d7000)] - 0x0000000af9945800 JavaThread "map-output-dispatcher-6" daemon [_thread_blocked, id=61187, stack(0x00000001780e0000,0x00000001784e3000)] - 0x0000000af9945e00 JavaThread "map-output-dispatcher-7" daemon [_thread_blocked, id=49411, stack(0x00000001784ec000,0x00000001788ef000)] - 0x0000000af9974600 JavaThread "dispatcher-BlockManagerMaster" daemon [_thread_blocked, id=60675, stack(0x00000001788f8000,0x0000000178cfb000)] - 0x0000000af9976400 JavaThread "dispatcher-BlockManagerEndpoint1" daemon [_thread_blocked, id=60163, stack(0x0000000178d04000,0x0000000179107000)] - 0x0000000af9976a00 JavaThread "RemoteBlock-temp-file-clean-thread" daemon [_thread_blocked, id=59907, stack(0x0000000179110000,0x0000000179513000)] - 0x0000000af9977600 JavaThread "dispatcher-HeartbeatReceiver" daemon [_thread_blocked, id=59395, stack(0x000000017951c000,0x000000017991f000)] - 0x0000000af9801200 JavaThread "heartbeat-receiver-event-loop-thread" daemon [_thread_blocked, id=49923, stack(0x0000000179928000,0x0000000179d2b000)] - 0x0000000af9ebb000 JavaThread "netty-rpc-env-timeout" daemon [_thread_blocked, id=58627, stack(0x0000000179d34000,0x000000017a137000)] - 0x0000000afcbcde00 JavaThread "task-starvation-timer" daemon [_thread_blocked, id=58115, stack(0x000000017a140000,0x000000017a543000)] - 0x0000000afcbcd800 JavaThread "task-abort-timer" daemon [_thread_blocked, id=50179, stack(0x000000017a54c000,0x000000017a94f000)] - 0x0000000afcd83600 JavaThread "dag-scheduler-event-loop" daemon [_thread_blocked, id=50691, stack(0x000000017a958000,0x000000017ad5b000)] - 0x0000000af99d8000 JavaThread "driver-heartbeater" daemon [_thread_blocked, id=57347, stack(0x000000017ad64000,0x000000017b167000)] - 0x0000000af99d9800 JavaThread "executor-kill-mark-cleanup" daemon [_thread_blocked, id=51203, stack(0x000000017b170000,0x000000017b573000)] - 0x0000000af99d9e00 JavaThread "executor-heartbeater" daemon [_thread_blocked, id=57091, stack(0x000000017b57c000,0x000000017b97f000)] - 0x0000000af99daa00 JavaThread "shuffle-boss-6-1" daemon [_thread_in_native, id=56835, stack(0x000000017b988000,0x000000017bd8b000)] - 0x0000000af99ec000 JavaThread "Spark Context Cleaner" daemon [_thread_blocked, id=52227, stack(0x000000017bd94000,0x000000017c197000)] - 0x0000000af99ec600 JavaThread "context-cleaner-periodic-gc" daemon [_thread_blocked, id=52739, stack(0x000000017c1a0000,0x000000017c5a3000)] - 0x0000000af99ecc00 JavaThread "spark-listener-group-appStatus" daemon [_thread_blocked, id=56323, stack(0x000000017c5ac000,0x000000017c9af000)] - 0x0000000af99ed200 JavaThread "spark-listener-group-executorManagement" daemon [_thread_blocked, id=52995, stack(0x000000017c9b8000,0x000000017cdbb000)] - 0x0000000af9a89800 JavaThread "element-tracking-store-worker" daemon [_thread_blocked, id=55811, stack(0x000000017cdc4000,0x000000017d1c7000)] - 0x0000000af9a8b600 JavaThread "spark-listener-group-shared" daemon [_thread_blocked, id=55299, stack(0x000000017d1d0000,0x000000017d5d3000)] - 0x0000000af9ad0600 JavaThread "Timer-0" [_thread_blocked, id=54787, stack(0x000000017d5dc000,0x000000017d9df000)] - 0x0000000af9bb8000 JavaThread "org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner" daemon [_thread_blocked, id=54531, stack(0x000000017d9e8000,0x000000017ddeb000)] - 0x0000000af9145800 JavaThread "process reaper" daemon [_thread_blocked, id=54275, stack(0x000000017ddf4000,0x000000017de2b000)] - 0x0000000afb041200 JavaThread "Executor task launch worker for task 28.0 in stage 4.0 (TID 11)" daemon [_thread_in_native, id=86535, stack(0x000000017e458000,0x000000017e85b000)] - 0x0000000afb041800 JavaThread "Executor task launch worker for task 31.0 in stage 4.0 (TID 42)" daemon [_thread_in_vm, id=86031, stack(0x000000017e864000,0x000000017ec67000)] -=>0x0000000afb041e00 JavaThread "Executor task launch worker for task 29.0 in stage 4.0 (TID 40)" daemon [_thread_in_vm, id=66563, stack(0x000000017ec70000,0x000000017f073000)] - 0x0000000afb042400 JavaThread "Executor task launch worker for task 30.0 in stage 4.0 (TID 41)" daemon [_thread_in_vm, id=85507, stack(0x000000017f07c000,0x000000017f47f000)] - 0x0000000afb042a00 JavaThread "Executor task launch worker for task 32.0 in stage 4.0 (TID 43)" daemon [_thread_in_vm, id=84995, stack(0x000000017f488000,0x000000017f88b000)] - 0x0000000afb24ea00 JavaThread "process reaper" daemon [_thread_blocked, id=83971, stack(0x000000016f608000,0x000000016f63f000)] - 0x0000000af8168000 JavaThread "process reaper" daemon [_thread_blocked, id=83715, stack(0x000000016f648000,0x000000016f67f000)] - 0x0000000af8168600 JavaThread "process reaper" daemon [_thread_blocked, id=68867, stack(0x000000017feb8000,0x000000017feef000)] - 0x0000000af8169200 JavaThread "process reaper" daemon [_thread_blocked, id=69123, stack(0x000000017fef8000,0x000000017ff2f000)] - 0x0000000afb503600 JavaThread "task-result-getter-0" daemon [_thread_blocked, id=45583, stack(0x000000017de34000,0x000000017e237000)] - 0x0000000af816a400 JavaThread "task-result-getter-1" daemon [_thread_blocked, id=65571, stack(0x000000017faa0000,0x000000017fea3000)] - 0x0000000afb5b8000 JavaThread "task-result-getter-2" daemon [_thread_blocked, id=87051, stack(0x0000000300004000,0x0000000300407000)] - 0x0000000af943d800 JavaThread "task-result-getter-3" daemon [_thread_blocked, id=67591, stack(0x0000000300410000,0x0000000300813000)] - 0x0000000af6862a00 JavaThread "block-manager-storage-async-thread-pool-0" daemon [_thread_blocked, id=69427, stack(0x0000000300e40000,0x0000000301243000)] - 0x0000000af6863000 JavaThread "block-manager-storage-async-thread-pool-1" daemon [_thread_blocked, id=70147, stack(0x0000000301658000,0x0000000301a5b000)] - 0x0000000af6863600 JavaThread "block-manager-storage-async-thread-pool-2" daemon [_thread_blocked, id=70659, stack(0x0000000301a64000,0x0000000301e67000)] - 0x0000000af99d9200 JavaThread "block-manager-ask-thread-pool-0" daemon [_thread_blocked, id=81411, stack(0x0000000301e70000,0x0000000302273000)] - 0x0000000afbfe4c00 JavaThread "block-manager-ask-thread-pool-1" daemon [_thread_blocked, id=71171, stack(0x000000030227c000,0x000000030267f000)] - 0x0000000afb24c000 JavaThread "block-manager-storage-async-thread-pool-3" daemon [_thread_blocked, id=82243, stack(0x000000030124c000,0x000000030164f000)] - 0x0000000af6861e00 JavaThread "block-manager-storage-async-thread-pool-4" daemon [_thread_blocked, id=81167, stack(0x0000000302688000,0x0000000302a8b000)] - 0x0000000af6e1c000 JavaThread "block-manager-storage-async-thread-pool-5" daemon [_thread_blocked, id=80643, stack(0x0000000302a94000,0x0000000302e97000)] - 0x0000000af6e1c600 JavaThread "block-manager-ask-thread-pool-2" daemon [_thread_blocked, id=80387, stack(0x0000000302ea0000,0x00000003032a3000)] - 0x0000000af6e1cc00 JavaThread "block-manager-ask-thread-pool-3" daemon [_thread_blocked, id=80131, stack(0x00000003032ac000,0x00000003036af000)] - 0x0000000af68a7600 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=73491, stack(0x0000000173ff0000,0x00000001741f3000)] - 0x0000000af8169800 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=73999, stack(0x00000001741fc000,0x00000001743ff000)] - 0x0000000af816aa00 JavaThread "C1 CompilerThread1" daemon [_thread_blocked, id=78099, stack(0x0000000174c38000,0x0000000174e3b000)] - 0x0000000afb470c00 JavaThread "C1 CompilerThread2" daemon [_thread_blocked, id=37403, stack(0x0000000174e44000,0x0000000175047000)] - 0x0000000afb471200 JavaThread "C1 CompilerThread3" daemon [_thread_blocked, id=45379, stack(0x000000017e240000,0x000000017e443000)] - 0x0000000af6860c00 JavaThread "QueryStageCreator-0" daemon [_thread_blocked, id=25155, stack(0x000000030081c000,0x0000000300c1f000)] - 0x0000000af5631e00 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=45091, stack(0x000000017f894000,0x000000017fa97000)] - 0x0000000af601c600 JavaThread "C2 CompilerThread4" daemon [_thread_blocked, id=67867, stack(0x0000000300c28000,0x0000000300e2b000)] - 0x0000000af601cc00 JavaThread "C2 CompilerThread5" daemon [_thread_blocked, id=69987, stack(0x00000003040f4000,0x00000003042f7000)] - -Other Threads: - 0x0000000102971070 VMThread "VM Thread" [stack: 0x000000016eb58000,0x000000016ed5b000] [id=20483] - 0x0000000afcb30300 WatcherThread [stack: 0x00000001712f4000,0x00000001714f7000] [id=28675] - 0x0000000102969bf0 GCTaskThread "GC Thread#0" [stack: 0x000000016e11c000,0x000000016e31f000] [id=12803] - 0x0000000afc3f0c00 GCTaskThread "GC Thread#1" [stack: 0x0000000171500000,0x0000000171703000] [id=32771] - 0x0000000afc3f0f00 GCTaskThread "GC Thread#2" [stack: 0x000000017170c000,0x000000017190f000] [id=43267] - 0x0000000afc3f1200 GCTaskThread "GC Thread#3" [stack: 0x0000000171918000,0x0000000171b1b000] [id=42755] - 0x0000000afc3f1500 GCTaskThread "GC Thread#4" [stack: 0x0000000171b24000,0x0000000171d27000] [id=42499] - 0x0000000afc3f1800 GCTaskThread "GC Thread#5" [stack: 0x0000000171d30000,0x0000000171f33000] [id=41987] - 0x0000000afc3f1b00 GCTaskThread "GC Thread#6" [stack: 0x0000000171f3c000,0x000000017213f000] [id=33539] - 0x0000000afc3f1e00 GCTaskThread "GC Thread#7" [stack: 0x0000000172148000,0x000000017234b000] [id=41475] - 0x0000000afc3f2100 GCTaskThread "GC Thread#8" [stack: 0x0000000172354000,0x0000000172557000] [id=41219] - 0x0000000afc3f2400 GCTaskThread "GC Thread#9" [stack: 0x0000000172560000,0x0000000172763000] [id=40963] - 0x0000000afc3f2700 GCTaskThread "GC Thread#10" [stack: 0x000000017276c000,0x000000017296f000] [id=34563] - 0x0000000afc3f2a00 GCTaskThread "GC Thread#11" [stack: 0x0000000172978000,0x0000000172b7b000] [id=34819] - 0x0000000afc3f2d00 GCTaskThread "GC Thread#12" [stack: 0x0000000172b84000,0x0000000172d87000] [id=35075] - 0x0000000afc3f3000 GCTaskThread "GC Thread#13" [stack: 0x0000000172d90000,0x0000000172f93000] [id=39939] - 0x0000000afc3f3300 GCTaskThread "GC Thread#14" [stack: 0x0000000172f9c000,0x000000017319f000] [id=39683] - 0x0000000afc3f3600 GCTaskThread "GC Thread#15" [stack: 0x00000001731a8000,0x00000001733ab000] [id=35843] - 0x0000000afc3f3900 GCTaskThread "GC Thread#16" [stack: 0x00000001733b4000,0x00000001735b7000] [id=36099] - 0x0000000afc3f3c00 GCTaskThread "GC Thread#17" [stack: 0x00000001735c0000,0x00000001737c3000] [id=38915] - 0x0000000afbcc4000 GCTaskThread "GC Thread#18" [stack: 0x00000001737cc000,0x00000001739cf000] [id=36611] - 0x0000000afbcc4300 GCTaskThread "GC Thread#19" [stack: 0x00000001739d8000,0x0000000173bdb000] [id=38147] - 0x000000010296a410 ConcurrentGCThread "G1 Main Marker" [stack: 0x000000016e328000,0x000000016e52b000] [id=13059] - 0x000000010296ad10 ConcurrentGCThread "G1 Conc#0" [stack: 0x000000016e534000,0x000000016e737000] [id=13827] - 0x0000000afbff7c00 ConcurrentGCThread "G1 Conc#1" [stack: 0x0000000174408000,0x000000017460b000] [id=65283] - 0x0000000afb9b4000 ConcurrentGCThread "G1 Conc#2" [stack: 0x0000000174614000,0x0000000174817000] [id=44291] - 0x0000000afb9b4300 ConcurrentGCThread "G1 Conc#3" [stack: 0x0000000174820000,0x0000000174a23000] [id=64771] - 0x0000000afb9b4600 ConcurrentGCThread "G1 Conc#4" [stack: 0x0000000174a2c000,0x0000000174c2f000] [id=44803] - 0x000000010296cc70 ConcurrentGCThread "G1 Refine#0" [stack: 0x000000016e740000,0x000000016e943000] [id=16643] - 0x000000010296d370 ConcurrentGCThread "G1 Service" [stack: 0x000000016e94c000,0x000000016eb4f000] [id=21507] - -Threads with active compile tasks: - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x00000004d5000000, size: 4096 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x000000d000000000-0x000000d000bc0000-0x000000d000bc0000), size 12320768, SharedBaseAddress: 0x000000d000000000, ArchiveRelocationMode: 1. -Compressed class space mapped at: 0x000000d001000000-0x000000d041000000, reserved size: 1073741824 -Narrow klass base: 0x000000d000000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 28 total, 28 available - Memory: 98304M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 1536M - Heap Max Capacity: 4G - Pre-touch: Disabled - Parallel Workers: 20 - Concurrent Workers: 5 - Concurrent Refinement Workers: 20 - Periodic GC: Disabled - -Heap: - garbage-first heap total 958464K, used 429201K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 115 young (235520K), 2 survivors (4096K) - Metaspace used 151232K, committed 152320K, reserved 1245184K - class space used 18131K, committed 18624K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x00000004d5000000, 0x00000004d5200000, 0x00000004d5200000|100%|HS| |TAMS 0x00000004d5200000, 0x00000004d5000000| Complete -| 1|0x00000004d5200000, 0x00000004d5400000, 0x00000004d5400000|100%|HS| |TAMS 0x00000004d5400000, 0x00000004d5200000| Complete -| 2|0x00000004d5400000, 0x00000004d5600000, 0x00000004d5600000|100%|HC| |TAMS 0x00000004d5600000, 0x00000004d5400000| Complete -| 3|0x00000004d5600000, 0x00000004d5800000, 0x00000004d5800000|100%|HS| |TAMS 0x00000004d5800000, 0x00000004d5600000| Complete -| 4|0x00000004d5800000, 0x00000004d5a00000, 0x00000004d5a00000|100%|HC| |TAMS 0x00000004d5a00000, 0x00000004d5800000| Complete -| 5|0x00000004d5a00000, 0x00000004d5c00000, 0x00000004d5c00000|100%|HC| |TAMS 0x00000004d5c00000, 0x00000004d5a00000| Complete -| 6|0x00000004d5c00000, 0x00000004d5e00000, 0x00000004d5e00000|100%|HC| |TAMS 0x00000004d5e00000, 0x00000004d5c00000| Complete -| 7|0x00000004d5e00000, 0x00000004d6000000, 0x00000004d6000000|100%|HC| |TAMS 0x00000004d6000000, 0x00000004d5e00000| Complete -| 8|0x00000004d6000000, 0x00000004d6200000, 0x00000004d6200000|100%|HC| |TAMS 0x00000004d6200000, 0x00000004d6000000| Complete -| 9|0x00000004d6200000, 0x00000004d6400000, 0x00000004d6400000|100%|HC| |TAMS 0x00000004d6400000, 0x00000004d6200000| Complete -| 10|0x00000004d6400000, 0x00000004d6600000, 0x00000004d6600000|100%|HC| |TAMS 0x00000004d6600000, 0x00000004d6400000| Complete -| 11|0x00000004d6600000, 0x00000004d6800000, 0x00000004d6800000|100%|HS| |TAMS 0x00000004d6800000, 0x00000004d6600000| Complete -| 12|0x00000004d6800000, 0x00000004d6a00000, 0x00000004d6a00000|100%|HS| |TAMS 0x00000004d6a00000, 0x00000004d6800000| Complete -| 13|0x00000004d6a00000, 0x00000004d6c00000, 0x00000004d6c00000|100%|HC| |TAMS 0x00000004d6c00000, 0x00000004d6a00000| Complete -| 14|0x00000004d6c00000, 0x00000004d6e00000, 0x00000004d6e00000|100%|HS| |TAMS 0x00000004d6e00000, 0x00000004d6c00000| Complete -| 15|0x00000004d6e00000, 0x00000004d7000000, 0x00000004d7000000|100%|HC| |TAMS 0x00000004d7000000, 0x00000004d6e00000| Complete -| 16|0x00000004d7000000, 0x00000004d7200000, 0x00000004d7200000|100%|HS| |TAMS 0x00000004d7200000, 0x00000004d7000000| Complete -| 17|0x00000004d7200000, 0x00000004d7400000, 0x00000004d7400000|100%|HS| |TAMS 0x00000004d7400000, 0x00000004d7200000| Complete -| 18|0x00000004d7400000, 0x00000004d7600000, 0x00000004d7600000|100%|HC| |TAMS 0x00000004d7600000, 0x00000004d7400000| Complete -| 19|0x00000004d7600000, 0x00000004d7800000, 0x00000004d7800000|100%| O| |TAMS 0x00000004d7800000, 0x00000004d7600000| Untracked -| 20|0x00000004d7800000, 0x00000004d7a00000, 0x00000004d7a00000|100%| O| |TAMS 0x00000004d7a00000, 0x00000004d7800000| Untracked -| 21|0x00000004d7a00000, 0x00000004d7c00000, 0x00000004d7c00000|100%| O| |TAMS 0x00000004d7c00000, 0x00000004d7a00000| Untracked -| 22|0x00000004d7c00000, 0x00000004d7e00000, 0x00000004d7e00000|100%| O| |TAMS 0x00000004d7e00000, 0x00000004d7c00000| Untracked -| 23|0x00000004d7e00000, 0x00000004d8000000, 0x00000004d8000000|100%| O| |TAMS 0x00000004d8000000, 0x00000004d7e00000| Untracked -| 24|0x00000004d8000000, 0x00000004d8200000, 0x00000004d8200000|100%| O| |TAMS 0x00000004d8200000, 0x00000004d8000000| Untracked -| 25|0x00000004d8200000, 0x00000004d8400000, 0x00000004d8400000|100%| O| |TAMS 0x00000004d8400000, 0x00000004d8200000| Untracked -| 26|0x00000004d8400000, 0x00000004d8600000, 0x00000004d8600000|100%| O| |TAMS 0x00000004d8600000, 0x00000004d8400000| Untracked -| 27|0x00000004d8600000, 0x00000004d8800000, 0x00000004d8800000|100%| O| |TAMS 0x00000004d8800000, 0x00000004d8600000| Untracked -| 28|0x00000004d8800000, 0x00000004d8a00000, 0x00000004d8a00000|100%| O| |TAMS 0x00000004d8a00000, 0x00000004d8800000| Untracked -| 29|0x00000004d8a00000, 0x00000004d8c00000, 0x00000004d8c00000|100%| O| |TAMS 0x00000004d8c00000, 0x00000004d8a00000| Untracked -| 30|0x00000004d8c00000, 0x00000004d8e00000, 0x00000004d8e00000|100%| O| |TAMS 0x00000004d8e00000, 0x00000004d8c00000| Untracked -| 31|0x00000004d8e00000, 0x00000004d9000000, 0x00000004d9000000|100%| O| |TAMS 0x00000004d9000000, 0x00000004d8e00000| Untracked -| 32|0x00000004d9000000, 0x00000004d9200000, 0x00000004d9200000|100%| O| |TAMS 0x00000004d9200000, 0x00000004d9000000| Untracked -| 33|0x00000004d9200000, 0x00000004d9400000, 0x00000004d9400000|100%| O| |TAMS 0x00000004d9400000, 0x00000004d9200000| Untracked -| 34|0x00000004d9400000, 0x00000004d9600000, 0x00000004d9600000|100%| O| |TAMS 0x00000004d9600000, 0x00000004d9400000| Untracked -| 35|0x00000004d9600000, 0x00000004d9800000, 0x00000004d9800000|100%| O| |TAMS 0x00000004d9800000, 0x00000004d9600000| Untracked -| 36|0x00000004d9800000, 0x00000004d9a00000, 0x00000004d9a00000|100%| O| |TAMS 0x00000004d9a00000, 0x00000004d9800000| Untracked -| 37|0x00000004d9a00000, 0x00000004d9c00000, 0x00000004d9c00000|100%| O| |TAMS 0x00000004d9c00000, 0x00000004d9a00000| Untracked -| 38|0x00000004d9c00000, 0x00000004d9e00000, 0x00000004d9e00000|100%| O| |TAMS 0x00000004d9e00000, 0x00000004d9c00000| Untracked -| 39|0x00000004d9e00000, 0x00000004da000000, 0x00000004da000000|100%| O| |TAMS 0x00000004da000000, 0x00000004d9e00000| Untracked -| 40|0x00000004da000000, 0x00000004da200000, 0x00000004da200000|100%| O| |TAMS 0x00000004da200000, 0x00000004da000000| Untracked -| 41|0x00000004da200000, 0x00000004da400000, 0x00000004da400000|100%| O| |TAMS 0x00000004da400000, 0x00000004da200000| Untracked -| 42|0x00000004da400000, 0x00000004da600000, 0x00000004da600000|100%| O| |TAMS 0x00000004da400000, 0x00000004da400000| Untracked -| 43|0x00000004da600000, 0x00000004da800000, 0x00000004da800000|100%| O| |TAMS 0x00000004da800000, 0x00000004da600000| Untracked -| 44|0x00000004da800000, 0x00000004daa00000, 0x00000004daa00000|100%| O| |TAMS 0x00000004daa00000, 0x00000004da800000| Untracked -| 45|0x00000004daa00000, 0x00000004dac00000, 0x00000004dac00000|100%| O| |TAMS 0x00000004dac00000, 0x00000004daa00000| Untracked -| 46|0x00000004dac00000, 0x00000004dae00000, 0x00000004dae00000|100%| O| |TAMS 0x00000004dae00000, 0x00000004dac00000| Untracked -| 47|0x00000004dae00000, 0x00000004db000000, 0x00000004db000000|100%| O| |TAMS 0x00000004db000000, 0x00000004dae00000| Untracked -| 48|0x00000004db000000, 0x00000004db200000, 0x00000004db200000|100%| O| |TAMS 0x00000004db200000, 0x00000004db000000| Untracked -| 49|0x00000004db200000, 0x00000004db2b4e00, 0x00000004db400000| 35%| O| |TAMS 0x00000004db200000, 0x00000004db200000| Untracked -| 50|0x00000004db400000, 0x00000004db600000, 0x00000004db600000|100%|HS| |TAMS 0x00000004db400000, 0x00000004db400000| Complete -| 51|0x00000004db600000, 0x00000004db800000, 0x00000004db800000|100%|HS| |TAMS 0x00000004db600000, 0x00000004db600000| Complete -| 52|0x00000004db800000, 0x00000004dba00000, 0x00000004dba00000|100%|HS| |TAMS 0x00000004db800000, 0x00000004db800000| Complete -| 53|0x00000004dba00000, 0x00000004dbc00000, 0x00000004dbc00000|100%| O| |TAMS 0x00000004dbc00000, 0x00000004dba00000| Untracked -| 54|0x00000004dbc00000, 0x00000004dbe00000, 0x00000004dbe00000|100%| O| |TAMS 0x00000004dbe00000, 0x00000004dbc00000| Untracked -| 55|0x00000004dbe00000, 0x00000004dc000000, 0x00000004dc000000|100%| O| |TAMS 0x00000004dc000000, 0x00000004dbe00000| Untracked -| 56|0x00000004dc000000, 0x00000004dc200000, 0x00000004dc200000|100%| O| |TAMS 0x00000004dc200000, 0x00000004dc000000| Untracked -| 57|0x00000004dc200000, 0x00000004dc400000, 0x00000004dc400000|100%| O| |TAMS 0x00000004dc400000, 0x00000004dc200000| Untracked -| 58|0x00000004dc400000, 0x00000004dc600000, 0x00000004dc600000|100%|HS| |TAMS 0x00000004dc400000, 0x00000004dc400000| Complete -| 59|0x00000004dc600000, 0x00000004dc800000, 0x00000004dc800000|100%|HS| |TAMS 0x00000004dc600000, 0x00000004dc600000| Complete -| 60|0x00000004dc800000, 0x00000004dca00000, 0x00000004dca00000|100%|HS| |TAMS 0x00000004dc800000, 0x00000004dc800000| Complete -| 61|0x00000004dca00000, 0x00000004dcc00000, 0x00000004dcc00000|100%|HC| |TAMS 0x00000004dca00000, 0x00000004dca00000| Complete -| 62|0x00000004dcc00000, 0x00000004dce00000, 0x00000004dce00000|100%|HC| |TAMS 0x00000004dcc00000, 0x00000004dcc00000| Complete -| 63|0x00000004dce00000, 0x00000004dd000000, 0x00000004dd000000|100%| O| |TAMS 0x00000004dd000000, 0x00000004dce00000| Untracked -| 64|0x00000004dd000000, 0x00000004dd200000, 0x00000004dd200000|100%|HS| |TAMS 0x00000004dd000000, 0x00000004dd000000| Complete -| 65|0x00000004dd200000, 0x00000004dd400000, 0x00000004dd400000|100%| O| |TAMS 0x00000004dd2bb200, 0x00000004dd200000| Untracked -| 66|0x00000004dd400000, 0x00000004dd600000, 0x00000004dd600000|100%|HS| |TAMS 0x00000004dd400000, 0x00000004dd400000| Complete -| 67|0x00000004dd600000, 0x00000004dd800000, 0x00000004dd800000|100%|HS| |TAMS 0x00000004dd600000, 0x00000004dd600000| Complete -| 68|0x00000004dd800000, 0x00000004dda00000, 0x00000004dda00000|100%|HS| |TAMS 0x00000004dd800000, 0x00000004dd800000| Complete -| 69|0x00000004dda00000, 0x00000004ddc00000, 0x00000004ddc00000|100%| O| |TAMS 0x00000004ddc00000, 0x00000004dda00000| Untracked -| 70|0x00000004ddc00000, 0x00000004dde00000, 0x00000004dde00000|100%| O| |TAMS 0x00000004dde00000, 0x00000004ddc00000| Untracked -| 71|0x00000004dde00000, 0x00000004de000000, 0x00000004de000000|100%| O| |TAMS 0x00000004de000000, 0x00000004dde00000| Untracked -| 72|0x00000004de000000, 0x00000004de200000, 0x00000004de200000|100%|HS| |TAMS 0x00000004de000000, 0x00000004de000000| Complete -| 73|0x00000004de200000, 0x00000004de400000, 0x00000004de400000|100%|HS| |TAMS 0x00000004de200000, 0x00000004de200000| Complete -| 74|0x00000004de400000, 0x00000004de600000, 0x00000004de600000|100%|HS| |TAMS 0x00000004de400000, 0x00000004de400000| Complete -| 75|0x00000004de600000, 0x00000004de800000, 0x00000004de800000|100%|HS| |TAMS 0x00000004de600000, 0x00000004de600000| Complete -| 76|0x00000004de800000, 0x00000004dea00000, 0x00000004dea00000|100%|HS| |TAMS 0x00000004de800000, 0x00000004de800000| Complete -| 77|0x00000004dea00000, 0x00000004dec00000, 0x00000004dec00000|100%|HS| |TAMS 0x00000004dea00000, 0x00000004dea00000| Complete -| 78|0x00000004dec00000, 0x00000004dee00000, 0x00000004dee00000|100%|HS| |TAMS 0x00000004dec00000, 0x00000004dec00000| Complete -| 79|0x00000004dee00000, 0x00000004df000000, 0x00000004df000000|100%|HS| |TAMS 0x00000004dee00000, 0x00000004dee00000| Complete -| 80|0x00000004df000000, 0x00000004df200000, 0x00000004df200000|100%|HS| |TAMS 0x00000004df000000, 0x00000004df000000| Complete -| 81|0x00000004df200000, 0x00000004df400000, 0x00000004df400000|100%|HS| |TAMS 0x00000004df200000, 0x00000004df200000| Complete -| 82|0x00000004df400000, 0x00000004df600000, 0x00000004df600000|100%|HS| |TAMS 0x00000004df400000, 0x00000004df400000| Complete -| 83|0x00000004df600000, 0x00000004df800000, 0x00000004df800000|100%|HS| |TAMS 0x00000004df600000, 0x00000004df600000| Complete -| 84|0x00000004df800000, 0x00000004dfa00000, 0x00000004dfa00000|100%|HS| |TAMS 0x00000004df800000, 0x00000004df800000| Complete -| 85|0x00000004dfa00000, 0x00000004dfc00000, 0x00000004dfc00000|100%|HS| |TAMS 0x00000004dfa00000, 0x00000004dfa00000| Complete -| 86|0x00000004dfc00000, 0x00000004dfe00000, 0x00000004dfe00000|100%|HS| |TAMS 0x00000004dfc00000, 0x00000004dfc00000| Complete -| 87|0x00000004dfe00000, 0x00000004e0000000, 0x00000004e0000000|100%|HS| |TAMS 0x00000004dfe00000, 0x00000004dfe00000| Complete -| 88|0x00000004e0000000, 0x00000004e0200000, 0x00000004e0200000|100%|HS| |TAMS 0x00000004e0000000, 0x00000004e0000000| Complete -| 89|0x00000004e0200000, 0x00000004e0400000, 0x00000004e0400000|100%|HS| |TAMS 0x00000004e0200000, 0x00000004e0200000| Complete -| 90|0x00000004e0400000, 0x00000004e0600000, 0x00000004e0600000|100%|HS| |TAMS 0x00000004e0400000, 0x00000004e0400000| Complete -| 91|0x00000004e0600000, 0x00000004e0800000, 0x00000004e0800000|100%|HS| |TAMS 0x00000004e0600000, 0x00000004e0600000| Complete -| 92|0x00000004e0800000, 0x00000004e0a00000, 0x00000004e0a00000|100%|HS| |TAMS 0x00000004e0800000, 0x00000004e0800000| Complete -| 93|0x00000004e0a00000, 0x00000004e0c00000, 0x00000004e0c00000|100%|HS| |TAMS 0x00000004e0a00000, 0x00000004e0a00000| Complete -| 94|0x00000004e0c00000, 0x00000004e0e00000, 0x00000004e0e00000|100%|HS| |TAMS 0x00000004e0c00000, 0x00000004e0c00000| Complete -| 95|0x00000004e0e00000, 0x00000004e1000000, 0x00000004e1000000|100%|HS| |TAMS 0x00000004e0e00000, 0x00000004e0e00000| Complete -| 96|0x00000004e1000000, 0x00000004e1000000, 0x00000004e1200000| 0%| F| |TAMS 0x00000004e1000000, 0x00000004e1000000| Untracked -| 97|0x00000004e1200000, 0x00000004e1200000, 0x00000004e1400000| 0%| F| |TAMS 0x00000004e1200000, 0x00000004e1200000| Untracked -| 98|0x00000004e1400000, 0x00000004e1400000, 0x00000004e1600000| 0%| F| |TAMS 0x00000004e1400000, 0x00000004e1400000| Untracked -| 99|0x00000004e1600000, 0x00000004e1600000, 0x00000004e1800000| 0%| F| |TAMS 0x00000004e1600000, 0x00000004e1600000| Untracked -| 100|0x00000004e1800000, 0x00000004e1800000, 0x00000004e1a00000| 0%| F| |TAMS 0x00000004e1800000, 0x00000004e1800000| Untracked -| 101|0x00000004e1a00000, 0x00000004e1a00000, 0x00000004e1c00000| 0%| F| |TAMS 0x00000004e1a00000, 0x00000004e1a00000| Untracked -| 102|0x00000004e1c00000, 0x00000004e1c00000, 0x00000004e1e00000| 0%| F| |TAMS 0x00000004e1c00000, 0x00000004e1c00000| Untracked -| 103|0x00000004e1e00000, 0x00000004e1e00000, 0x00000004e2000000| 0%| F| |TAMS 0x00000004e1e00000, 0x00000004e1e00000| Untracked -| 104|0x00000004e2000000, 0x00000004e2000000, 0x00000004e2200000| 0%| F| |TAMS 0x00000004e2000000, 0x00000004e2000000| Untracked -| 105|0x00000004e2200000, 0x00000004e2200000, 0x00000004e2400000| 0%| F| |TAMS 0x00000004e2200000, 0x00000004e2200000| Untracked -| 106|0x00000004e2400000, 0x00000004e2400000, 0x00000004e2600000| 0%| F| |TAMS 0x00000004e2400000, 0x00000004e2400000| Untracked -| 107|0x00000004e2600000, 0x00000004e2600000, 0x00000004e2800000| 0%| F| |TAMS 0x00000004e2600000, 0x00000004e2600000| Untracked -| 108|0x00000004e2800000, 0x00000004e2800000, 0x00000004e2a00000| 0%| F| |TAMS 0x00000004e2800000, 0x00000004e2800000| Untracked -| 109|0x00000004e2a00000, 0x00000004e2a00000, 0x00000004e2c00000| 0%| F| |TAMS 0x00000004e2a00000, 0x00000004e2a00000| Untracked -| 110|0x00000004e2c00000, 0x00000004e2c00000, 0x00000004e2e00000| 0%| F| |TAMS 0x00000004e2c00000, 0x00000004e2c00000| Untracked -| 111|0x00000004e2e00000, 0x00000004e2e00000, 0x00000004e3000000| 0%| F| |TAMS 0x00000004e2e00000, 0x00000004e2e00000| Untracked -| 112|0x00000004e3000000, 0x00000004e3000000, 0x00000004e3200000| 0%| F| |TAMS 0x00000004e3000000, 0x00000004e3000000| Untracked -| 113|0x00000004e3200000, 0x00000004e3200000, 0x00000004e3400000| 0%| F| |TAMS 0x00000004e3200000, 0x00000004e3200000| Untracked -| 114|0x00000004e3400000, 0x00000004e3400000, 0x00000004e3600000| 0%| F| |TAMS 0x00000004e3400000, 0x00000004e3400000| Untracked -| 115|0x00000004e3600000, 0x00000004e3600000, 0x00000004e3800000| 0%| F| |TAMS 0x00000004e3600000, 0x00000004e3600000| Untracked -| 116|0x00000004e3800000, 0x00000004e3800000, 0x00000004e3a00000| 0%| F| |TAMS 0x00000004e3800000, 0x00000004e3800000| Untracked -| 117|0x00000004e3a00000, 0x00000004e3a00000, 0x00000004e3c00000| 0%| F| |TAMS 0x00000004e3a00000, 0x00000004e3a00000| Untracked -| 118|0x00000004e3c00000, 0x00000004e3c00000, 0x00000004e3e00000| 0%| F| |TAMS 0x00000004e3c00000, 0x00000004e3c00000| Untracked -| 119|0x00000004e3e00000, 0x00000004e3e00000, 0x00000004e4000000| 0%| F| |TAMS 0x00000004e3e00000, 0x00000004e3e00000| Untracked -| 120|0x00000004e4000000, 0x00000004e4000000, 0x00000004e4200000| 0%| F| |TAMS 0x00000004e4000000, 0x00000004e4000000| Untracked -| 121|0x00000004e4200000, 0x00000004e4200000, 0x00000004e4400000| 0%| F| |TAMS 0x00000004e4200000, 0x00000004e4200000| Untracked -| 122|0x00000004e4400000, 0x00000004e4400000, 0x00000004e4600000| 0%| F| |TAMS 0x00000004e4400000, 0x00000004e4400000| Untracked -| 123|0x00000004e4600000, 0x00000004e4600000, 0x00000004e4800000| 0%| F| |TAMS 0x00000004e4600000, 0x00000004e4600000| Untracked -| 124|0x00000004e4800000, 0x00000004e4800000, 0x00000004e4a00000| 0%| F| |TAMS 0x00000004e4800000, 0x00000004e4800000| Untracked -| 125|0x00000004e4a00000, 0x00000004e4a00000, 0x00000004e4c00000| 0%| F| |TAMS 0x00000004e4a00000, 0x00000004e4a00000| Untracked -| 126|0x00000004e4c00000, 0x00000004e4c00000, 0x00000004e4e00000| 0%| F| |TAMS 0x00000004e4c00000, 0x00000004e4c00000| Untracked -| 127|0x00000004e4e00000, 0x00000004e4e00000, 0x00000004e5000000| 0%| F| |TAMS 0x00000004e4e00000, 0x00000004e4e00000| Untracked -| 128|0x00000004e5000000, 0x00000004e5000000, 0x00000004e5200000| 0%| F| |TAMS 0x00000004e5000000, 0x00000004e5000000| Untracked -| 129|0x00000004e5200000, 0x00000004e5200000, 0x00000004e5400000| 0%| F| |TAMS 0x00000004e5200000, 0x00000004e5200000| Untracked -| 130|0x00000004e5400000, 0x00000004e5400000, 0x00000004e5600000| 0%| F| |TAMS 0x00000004e5400000, 0x00000004e5400000| Untracked -| 131|0x00000004e5600000, 0x00000004e5600000, 0x00000004e5800000| 0%| F| |TAMS 0x00000004e5600000, 0x00000004e5600000| Untracked -| 132|0x00000004e5800000, 0x00000004e5800000, 0x00000004e5a00000| 0%| F| |TAMS 0x00000004e5800000, 0x00000004e5800000| Untracked -| 133|0x00000004e5a00000, 0x00000004e5a00000, 0x00000004e5c00000| 0%| F| |TAMS 0x00000004e5a00000, 0x00000004e5a00000| Untracked -| 134|0x00000004e5c00000, 0x00000004e5c00000, 0x00000004e5e00000| 0%| F| |TAMS 0x00000004e5c00000, 0x00000004e5c00000| Untracked -| 135|0x00000004e5e00000, 0x00000004e5e00000, 0x00000004e6000000| 0%| F| |TAMS 0x00000004e5e00000, 0x00000004e5e00000| Untracked -| 136|0x00000004e6000000, 0x00000004e6000000, 0x00000004e6200000| 0%| F| |TAMS 0x00000004e6000000, 0x00000004e6000000| Untracked -| 137|0x00000004e6200000, 0x00000004e6200000, 0x00000004e6400000| 0%| F| |TAMS 0x00000004e6200000, 0x00000004e6200000| Untracked -| 138|0x00000004e6400000, 0x00000004e6400000, 0x00000004e6600000| 0%| F| |TAMS 0x00000004e6400000, 0x00000004e6400000| Untracked -| 139|0x00000004e6600000, 0x00000004e6600000, 0x00000004e6800000| 0%| F| |TAMS 0x00000004e6600000, 0x00000004e6600000| Untracked -| 140|0x00000004e6800000, 0x00000004e6800000, 0x00000004e6a00000| 0%| F| |TAMS 0x00000004e6800000, 0x00000004e6800000| Untracked -| 141|0x00000004e6a00000, 0x00000004e6a00000, 0x00000004e6c00000| 0%| F| |TAMS 0x00000004e6a00000, 0x00000004e6a00000| Untracked -| 142|0x00000004e6c00000, 0x00000004e6c00000, 0x00000004e6e00000| 0%| F| |TAMS 0x00000004e6c00000, 0x00000004e6c00000| Untracked -| 143|0x00000004e6e00000, 0x00000004e6e00000, 0x00000004e7000000| 0%| F| |TAMS 0x00000004e6e00000, 0x00000004e6e00000| Untracked -| 144|0x00000004e7000000, 0x00000004e7000000, 0x00000004e7200000| 0%| F| |TAMS 0x00000004e7000000, 0x00000004e7000000| Untracked -| 145|0x00000004e7200000, 0x00000004e7200000, 0x00000004e7400000| 0%| F| |TAMS 0x00000004e7200000, 0x00000004e7200000| Untracked -| 146|0x00000004e7400000, 0x00000004e7400000, 0x00000004e7600000| 0%| F| |TAMS 0x00000004e7400000, 0x00000004e7400000| Untracked -| 147|0x00000004e7600000, 0x00000004e7600000, 0x00000004e7800000| 0%| F| |TAMS 0x00000004e7600000, 0x00000004e7600000| Untracked -| 148|0x00000004e7800000, 0x00000004e7800000, 0x00000004e7a00000| 0%| F| |TAMS 0x00000004e7800000, 0x00000004e7800000| Untracked -| 149|0x00000004e7a00000, 0x00000004e7a00000, 0x00000004e7c00000| 0%| F| |TAMS 0x00000004e7a00000, 0x00000004e7a00000| Untracked -| 150|0x00000004e7c00000, 0x00000004e7c00000, 0x00000004e7e00000| 0%| F| |TAMS 0x00000004e7c00000, 0x00000004e7c00000| Untracked -| 151|0x00000004e7e00000, 0x00000004e7e00000, 0x00000004e8000000| 0%| F| |TAMS 0x00000004e7e00000, 0x00000004e7e00000| Untracked -| 152|0x00000004e8000000, 0x00000004e8000000, 0x00000004e8200000| 0%| F| |TAMS 0x00000004e8000000, 0x00000004e8000000| Untracked -| 153|0x00000004e8200000, 0x00000004e8200000, 0x00000004e8400000| 0%| F| |TAMS 0x00000004e8200000, 0x00000004e8200000| Untracked -| 154|0x00000004e8400000, 0x00000004e8400000, 0x00000004e8600000| 0%| F| |TAMS 0x00000004e8400000, 0x00000004e8400000| Untracked -| 155|0x00000004e8600000, 0x00000004e8600000, 0x00000004e8800000| 0%| F| |TAMS 0x00000004e8600000, 0x00000004e8600000| Untracked -| 156|0x00000004e8800000, 0x00000004e8800000, 0x00000004e8a00000| 0%| F| |TAMS 0x00000004e8800000, 0x00000004e8800000| Untracked -| 157|0x00000004e8a00000, 0x00000004e8a00000, 0x00000004e8c00000| 0%| F| |TAMS 0x00000004e8a00000, 0x00000004e8a00000| Untracked -| 158|0x00000004e8c00000, 0x00000004e8c00000, 0x00000004e8e00000| 0%| F| |TAMS 0x00000004e8c00000, 0x00000004e8c00000| Untracked -| 159|0x00000004e8e00000, 0x00000004e8e00000, 0x00000004e9000000| 0%| F| |TAMS 0x00000004e8e00000, 0x00000004e8e00000| Untracked -| 160|0x00000004e9000000, 0x00000004e9000000, 0x00000004e9200000| 0%| F| |TAMS 0x00000004e9000000, 0x00000004e9000000| Untracked -| 161|0x00000004e9200000, 0x00000004e9200000, 0x00000004e9400000| 0%| F| |TAMS 0x00000004e9200000, 0x00000004e9200000| Untracked -| 162|0x00000004e9400000, 0x00000004e9400000, 0x00000004e9600000| 0%| F| |TAMS 0x00000004e9400000, 0x00000004e9400000| Untracked -| 163|0x00000004e9600000, 0x00000004e9600000, 0x00000004e9800000| 0%| F| |TAMS 0x00000004e9600000, 0x00000004e9600000| Untracked -| 164|0x00000004e9800000, 0x00000004e9800000, 0x00000004e9a00000| 0%| F| |TAMS 0x00000004e9800000, 0x00000004e9800000| Untracked -| 165|0x00000004e9a00000, 0x00000004e9a00000, 0x00000004e9c00000| 0%| F| |TAMS 0x00000004e9a00000, 0x00000004e9a00000| Untracked -| 166|0x00000004e9c00000, 0x00000004e9c00000, 0x00000004e9e00000| 0%| F| |TAMS 0x00000004e9c00000, 0x00000004e9c00000| Untracked -| 167|0x00000004e9e00000, 0x00000004e9e00000, 0x00000004ea000000| 0%| F| |TAMS 0x00000004e9e00000, 0x00000004e9e00000| Untracked -| 168|0x00000004ea000000, 0x00000004ea000000, 0x00000004ea200000| 0%| F| |TAMS 0x00000004ea000000, 0x00000004ea000000| Untracked -| 169|0x00000004ea200000, 0x00000004ea200000, 0x00000004ea400000| 0%| F| |TAMS 0x00000004ea200000, 0x00000004ea200000| Untracked -| 170|0x00000004ea400000, 0x00000004ea400000, 0x00000004ea600000| 0%| F| |TAMS 0x00000004ea400000, 0x00000004ea400000| Untracked -| 171|0x00000004ea600000, 0x00000004ea600000, 0x00000004ea800000| 0%| F| |TAMS 0x00000004ea600000, 0x00000004ea600000| Untracked -| 172|0x00000004ea800000, 0x00000004ea800000, 0x00000004eaa00000| 0%| F| |TAMS 0x00000004ea800000, 0x00000004ea800000| Untracked -| 173|0x00000004eaa00000, 0x00000004eaa00000, 0x00000004eac00000| 0%| F| |TAMS 0x00000004eaa00000, 0x00000004eaa00000| Untracked -| 174|0x00000004eac00000, 0x00000004eac00000, 0x00000004eae00000| 0%| F| |TAMS 0x00000004eac00000, 0x00000004eac00000| Untracked -| 175|0x00000004eae00000, 0x00000004eae00000, 0x00000004eb000000| 0%| F| |TAMS 0x00000004eae00000, 0x00000004eae00000| Untracked -| 176|0x00000004eb000000, 0x00000004eb000000, 0x00000004eb200000| 0%| F| |TAMS 0x00000004eb000000, 0x00000004eb000000| Untracked -| 177|0x00000004eb200000, 0x00000004eb200000, 0x00000004eb400000| 0%| F| |TAMS 0x00000004eb200000, 0x00000004eb200000| Untracked -| 178|0x00000004eb400000, 0x00000004eb400000, 0x00000004eb600000| 0%| F| |TAMS 0x00000004eb400000, 0x00000004eb400000| Untracked -| 179|0x00000004eb600000, 0x00000004eb600000, 0x00000004eb800000| 0%| F| |TAMS 0x00000004eb600000, 0x00000004eb600000| Untracked -| 180|0x00000004eb800000, 0x00000004eb800000, 0x00000004eba00000| 0%| F| |TAMS 0x00000004eb800000, 0x00000004eb800000| Untracked -| 181|0x00000004eba00000, 0x00000004eba00000, 0x00000004ebc00000| 0%| F| |TAMS 0x00000004eba00000, 0x00000004eba00000| Untracked -| 182|0x00000004ebc00000, 0x00000004ebc00000, 0x00000004ebe00000| 0%| F| |TAMS 0x00000004ebc00000, 0x00000004ebc00000| Untracked -| 183|0x00000004ebe00000, 0x00000004ebe00000, 0x00000004ec000000| 0%| F| |TAMS 0x00000004ebe00000, 0x00000004ebe00000| Untracked -| 184|0x00000004ec000000, 0x00000004ec000000, 0x00000004ec200000| 0%| F| |TAMS 0x00000004ec000000, 0x00000004ec000000| Untracked -| 185|0x00000004ec200000, 0x00000004ec200000, 0x00000004ec400000| 0%| F| |TAMS 0x00000004ec200000, 0x00000004ec200000| Untracked -| 186|0x00000004ec400000, 0x00000004ec487668, 0x00000004ec600000| 26%| S|CS|TAMS 0x00000004ec400000, 0x00000004ec400000| Complete -| 187|0x00000004ec600000, 0x00000004ec800000, 0x00000004ec800000|100%| S|CS|TAMS 0x00000004ec600000, 0x00000004ec600000| Complete -| 188|0x00000004ec800000, 0x00000004ec800000, 0x00000004eca00000| 0%| F| |TAMS 0x00000004ec800000, 0x00000004ec800000| Untracked -| 189|0x00000004eca00000, 0x00000004eca00000, 0x00000004ecc00000| 0%| F| |TAMS 0x00000004eca00000, 0x00000004eca00000| Untracked -| 190|0x00000004ecc00000, 0x00000004ecc00000, 0x00000004ece00000| 0%| F| |TAMS 0x00000004ecc00000, 0x00000004ecc00000| Untracked -| 191|0x00000004ece00000, 0x00000004ece00000, 0x00000004ed000000| 0%| F| |TAMS 0x00000004ece00000, 0x00000004ece00000| Untracked -| 192|0x00000004ed000000, 0x00000004ed000000, 0x00000004ed200000| 0%| F| |TAMS 0x00000004ed000000, 0x00000004ed000000| Untracked -| 193|0x00000004ed200000, 0x00000004ed200000, 0x00000004ed400000| 0%| F| |TAMS 0x00000004ed200000, 0x00000004ed200000| Untracked -| 194|0x00000004ed400000, 0x00000004ed400000, 0x00000004ed600000| 0%| F| |TAMS 0x00000004ed400000, 0x00000004ed400000| Untracked -| 195|0x00000004ed600000, 0x00000004ed600000, 0x00000004ed800000| 0%| F| |TAMS 0x00000004ed600000, 0x00000004ed600000| Untracked -| 196|0x00000004ed800000, 0x00000004ed800000, 0x00000004eda00000| 0%| F| |TAMS 0x00000004ed800000, 0x00000004ed800000| Untracked -| 197|0x00000004eda00000, 0x00000004eda00000, 0x00000004edc00000| 0%| F| |TAMS 0x00000004eda00000, 0x00000004eda00000| Untracked -| 198|0x00000004edc00000, 0x00000004edc00000, 0x00000004ede00000| 0%| F| |TAMS 0x00000004edc00000, 0x00000004edc00000| Untracked -| 199|0x00000004ede00000, 0x00000004ede00000, 0x00000004ee000000| 0%| F| |TAMS 0x00000004ede00000, 0x00000004ede00000| Untracked -| 200|0x00000004ee000000, 0x00000004ee000000, 0x00000004ee200000| 0%| F| |TAMS 0x00000004ee000000, 0x00000004ee000000| Untracked -| 201|0x00000004ee200000, 0x00000004ee200000, 0x00000004ee400000| 0%| F| |TAMS 0x00000004ee200000, 0x00000004ee200000| Untracked -| 202|0x00000004ee400000, 0x00000004ee400000, 0x00000004ee600000| 0%| F| |TAMS 0x00000004ee400000, 0x00000004ee400000| Untracked -| 203|0x00000004ee600000, 0x00000004ee600000, 0x00000004ee800000| 0%| F| |TAMS 0x00000004ee600000, 0x00000004ee600000| Untracked -| 204|0x00000004ee800000, 0x00000004ee800000, 0x00000004eea00000| 0%| F| |TAMS 0x00000004ee800000, 0x00000004ee800000| Untracked -| 205|0x00000004eea00000, 0x00000004eea00000, 0x00000004eec00000| 0%| F| |TAMS 0x00000004eea00000, 0x00000004eea00000| Untracked -| 206|0x00000004eec00000, 0x00000004eec00000, 0x00000004eee00000| 0%| F| |TAMS 0x00000004eec00000, 0x00000004eec00000| Untracked -| 207|0x00000004eee00000, 0x00000004eee00000, 0x00000004ef000000| 0%| F| |TAMS 0x00000004eee00000, 0x00000004eee00000| Untracked -| 208|0x00000004ef000000, 0x00000004ef000000, 0x00000004ef200000| 0%| F| |TAMS 0x00000004ef000000, 0x00000004ef000000| Untracked -| 209|0x00000004ef200000, 0x00000004ef200000, 0x00000004ef400000| 0%| F| |TAMS 0x00000004ef200000, 0x00000004ef200000| Untracked -| 210|0x00000004ef400000, 0x00000004ef400000, 0x00000004ef600000| 0%| F| |TAMS 0x00000004ef400000, 0x00000004ef400000| Untracked -| 211|0x00000004ef600000, 0x00000004ef600000, 0x00000004ef800000| 0%| F| |TAMS 0x00000004ef600000, 0x00000004ef600000| Untracked -| 212|0x00000004ef800000, 0x00000004ef800000, 0x00000004efa00000| 0%| F| |TAMS 0x00000004ef800000, 0x00000004ef800000| Untracked -| 213|0x00000004efa00000, 0x00000004efa00000, 0x00000004efc00000| 0%| F| |TAMS 0x00000004efa00000, 0x00000004efa00000| Untracked -| 214|0x00000004efc00000, 0x00000004efc00000, 0x00000004efe00000| 0%| F| |TAMS 0x00000004efc00000, 0x00000004efc00000| Untracked -| 215|0x00000004efe00000, 0x00000004efe00000, 0x00000004f0000000| 0%| F| |TAMS 0x00000004efe00000, 0x00000004efe00000| Untracked -| 216|0x00000004f0000000, 0x00000004f0000000, 0x00000004f0200000| 0%| F| |TAMS 0x00000004f0000000, 0x00000004f0000000| Untracked -| 217|0x00000004f0200000, 0x00000004f0200000, 0x00000004f0400000| 0%| F| |TAMS 0x00000004f0200000, 0x00000004f0200000| Untracked -| 218|0x00000004f0400000, 0x00000004f0400000, 0x00000004f0600000| 0%| F| |TAMS 0x00000004f0400000, 0x00000004f0400000| Untracked -| 219|0x00000004f0600000, 0x00000004f0600000, 0x00000004f0800000| 0%| F| |TAMS 0x00000004f0600000, 0x00000004f0600000| Untracked -| 220|0x00000004f0800000, 0x00000004f0800000, 0x00000004f0a00000| 0%| F| |TAMS 0x00000004f0800000, 0x00000004f0800000| Untracked -| 221|0x00000004f0a00000, 0x00000004f0a00000, 0x00000004f0c00000| 0%| F| |TAMS 0x00000004f0a00000, 0x00000004f0a00000| Untracked -| 222|0x00000004f0c00000, 0x00000004f0c00000, 0x00000004f0e00000| 0%| F| |TAMS 0x00000004f0c00000, 0x00000004f0c00000| Untracked -| 223|0x00000004f0e00000, 0x00000004f0e00000, 0x00000004f1000000| 0%| F| |TAMS 0x00000004f0e00000, 0x00000004f0e00000| Untracked -| 224|0x00000004f1000000, 0x00000004f1000000, 0x00000004f1200000| 0%| F| |TAMS 0x00000004f1000000, 0x00000004f1000000| Untracked -| 225|0x00000004f1200000, 0x00000004f1200000, 0x00000004f1400000| 0%| F| |TAMS 0x00000004f1200000, 0x00000004f1200000| Untracked -| 226|0x00000004f1400000, 0x00000004f1400000, 0x00000004f1600000| 0%| F| |TAMS 0x00000004f1400000, 0x00000004f1400000| Untracked -| 227|0x00000004f1600000, 0x00000004f1600000, 0x00000004f1800000| 0%| F| |TAMS 0x00000004f1600000, 0x00000004f1600000| Untracked -| 228|0x00000004f1800000, 0x00000004f1800000, 0x00000004f1a00000| 0%| F| |TAMS 0x00000004f1800000, 0x00000004f1800000| Untracked -| 229|0x00000004f1a00000, 0x00000004f1a00000, 0x00000004f1c00000| 0%| F| |TAMS 0x00000004f1a00000, 0x00000004f1a00000| Untracked -| 230|0x00000004f1c00000, 0x00000004f1c00000, 0x00000004f1e00000| 0%| F| |TAMS 0x00000004f1c00000, 0x00000004f1c00000| Untracked -| 231|0x00000004f1e00000, 0x00000004f1e00000, 0x00000004f2000000| 0%| F| |TAMS 0x00000004f1e00000, 0x00000004f1e00000| Untracked -| 232|0x00000004f2000000, 0x00000004f2000000, 0x00000004f2200000| 0%| F| |TAMS 0x00000004f2000000, 0x00000004f2000000| Untracked -| 233|0x00000004f2200000, 0x00000004f2200000, 0x00000004f2400000| 0%| F| |TAMS 0x00000004f2200000, 0x00000004f2200000| Untracked -| 234|0x00000004f2400000, 0x00000004f2400000, 0x00000004f2600000| 0%| F| |TAMS 0x00000004f2400000, 0x00000004f2400000| Untracked -| 235|0x00000004f2600000, 0x00000004f2600000, 0x00000004f2800000| 0%| F| |TAMS 0x00000004f2600000, 0x00000004f2600000| Untracked -| 236|0x00000004f2800000, 0x00000004f2800000, 0x00000004f2a00000| 0%| F| |TAMS 0x00000004f2800000, 0x00000004f2800000| Untracked -| 237|0x00000004f2a00000, 0x00000004f2a00000, 0x00000004f2c00000| 0%| F| |TAMS 0x00000004f2a00000, 0x00000004f2a00000| Untracked -| 238|0x00000004f2c00000, 0x00000004f2c00000, 0x00000004f2e00000| 0%| F| |TAMS 0x00000004f2c00000, 0x00000004f2c00000| Untracked -| 239|0x00000004f2e00000, 0x00000004f2e00000, 0x00000004f3000000| 0%| F| |TAMS 0x00000004f2e00000, 0x00000004f2e00000| Untracked -| 240|0x00000004f3000000, 0x00000004f3000000, 0x00000004f3200000| 0%| F| |TAMS 0x00000004f3000000, 0x00000004f3000000| Untracked -| 241|0x00000004f3200000, 0x00000004f3200000, 0x00000004f3400000| 0%| F| |TAMS 0x00000004f3200000, 0x00000004f3200000| Untracked -| 242|0x00000004f3400000, 0x00000004f3400000, 0x00000004f3600000| 0%| F| |TAMS 0x00000004f3400000, 0x00000004f3400000| Untracked -| 243|0x00000004f3600000, 0x00000004f3600000, 0x00000004f3800000| 0%| F| |TAMS 0x00000004f3600000, 0x00000004f3600000| Untracked -| 244|0x00000004f3800000, 0x00000004f3800000, 0x00000004f3a00000| 0%| F| |TAMS 0x00000004f3800000, 0x00000004f3800000| Untracked -| 245|0x00000004f3a00000, 0x00000004f3a00000, 0x00000004f3c00000| 0%| F| |TAMS 0x00000004f3a00000, 0x00000004f3a00000| Untracked -| 246|0x00000004f3c00000, 0x00000004f3c00000, 0x00000004f3e00000| 0%| F| |TAMS 0x00000004f3c00000, 0x00000004f3c00000| Untracked -| 247|0x00000004f3e00000, 0x00000004f3e00000, 0x00000004f4000000| 0%| F| |TAMS 0x00000004f3e00000, 0x00000004f3e00000| Untracked -| 248|0x00000004f4000000, 0x00000004f4000000, 0x00000004f4200000| 0%| F| |TAMS 0x00000004f4000000, 0x00000004f4000000| Untracked -| 249|0x00000004f4200000, 0x00000004f4200000, 0x00000004f4400000| 0%| F| |TAMS 0x00000004f4200000, 0x00000004f4200000| Untracked -| 250|0x00000004f4400000, 0x00000004f4400000, 0x00000004f4600000| 0%| F| |TAMS 0x00000004f4400000, 0x00000004f4400000| Untracked -| 251|0x00000004f4600000, 0x00000004f4600000, 0x00000004f4800000| 0%| F| |TAMS 0x00000004f4600000, 0x00000004f4600000| Untracked -| 252|0x00000004f4800000, 0x00000004f4800000, 0x00000004f4a00000| 0%| F| |TAMS 0x00000004f4800000, 0x00000004f4800000| Untracked -| 253|0x00000004f4a00000, 0x00000004f4a00000, 0x00000004f4c00000| 0%| F| |TAMS 0x00000004f4a00000, 0x00000004f4a00000| Untracked -| 254|0x00000004f4c00000, 0x00000004f4c00000, 0x00000004f4e00000| 0%| F| |TAMS 0x00000004f4c00000, 0x00000004f4c00000| Untracked -| 255|0x00000004f4e00000, 0x00000004f4e00000, 0x00000004f5000000| 0%| F| |TAMS 0x00000004f4e00000, 0x00000004f4e00000| Untracked -| 256|0x00000004f5000000, 0x00000004f5000000, 0x00000004f5200000| 0%| F| |TAMS 0x00000004f5000000, 0x00000004f5000000| Untracked -| 257|0x00000004f5200000, 0x00000004f5200000, 0x00000004f5400000| 0%| F| |TAMS 0x00000004f5200000, 0x00000004f5200000| Untracked -| 258|0x00000004f5400000, 0x00000004f5400000, 0x00000004f5600000| 0%| F| |TAMS 0x00000004f5400000, 0x00000004f5400000| Untracked -| 259|0x00000004f5600000, 0x00000004f5600000, 0x00000004f5800000| 0%| F| |TAMS 0x00000004f5600000, 0x00000004f5600000| Untracked -| 260|0x00000004f5800000, 0x00000004f5800000, 0x00000004f5a00000| 0%| F| |TAMS 0x00000004f5800000, 0x00000004f5800000| Untracked -| 261|0x00000004f5a00000, 0x00000004f5a00000, 0x00000004f5c00000| 0%| F| |TAMS 0x00000004f5a00000, 0x00000004f5a00000| Untracked -| 262|0x00000004f5c00000, 0x00000004f5c00000, 0x00000004f5e00000| 0%| F| |TAMS 0x00000004f5c00000, 0x00000004f5c00000| Untracked -| 263|0x00000004f5e00000, 0x00000004f5e00000, 0x00000004f6000000| 0%| F| |TAMS 0x00000004f5e00000, 0x00000004f5e00000| Untracked -| 264|0x00000004f6000000, 0x00000004f6000000, 0x00000004f6200000| 0%| F| |TAMS 0x00000004f6000000, 0x00000004f6000000| Untracked -| 265|0x00000004f6200000, 0x00000004f6200000, 0x00000004f6400000| 0%| F| |TAMS 0x00000004f6200000, 0x00000004f6200000| Untracked -| 266|0x00000004f6400000, 0x00000004f6400000, 0x00000004f6600000| 0%| F| |TAMS 0x00000004f6400000, 0x00000004f6400000| Untracked -| 267|0x00000004f6600000, 0x00000004f6600000, 0x00000004f6800000| 0%| F| |TAMS 0x00000004f6600000, 0x00000004f6600000| Untracked -| 268|0x00000004f6800000, 0x00000004f6800000, 0x00000004f6a00000| 0%| F| |TAMS 0x00000004f6800000, 0x00000004f6800000| Untracked -| 269|0x00000004f6a00000, 0x00000004f6a00000, 0x00000004f6c00000| 0%| F| |TAMS 0x00000004f6a00000, 0x00000004f6a00000| Untracked -| 270|0x00000004f6c00000, 0x00000004f6c00000, 0x00000004f6e00000| 0%| F| |TAMS 0x00000004f6c00000, 0x00000004f6c00000| Untracked -| 271|0x00000004f6e00000, 0x00000004f6e00000, 0x00000004f7000000| 0%| F| |TAMS 0x00000004f6e00000, 0x00000004f6e00000| Untracked -| 272|0x00000004f7000000, 0x00000004f7000000, 0x00000004f7200000| 0%| F| |TAMS 0x00000004f7000000, 0x00000004f7000000| Untracked -| 273|0x00000004f7200000, 0x00000004f7200000, 0x00000004f7400000| 0%| F| |TAMS 0x00000004f7200000, 0x00000004f7200000| Untracked -| 274|0x00000004f7400000, 0x00000004f7400000, 0x00000004f7600000| 0%| F| |TAMS 0x00000004f7400000, 0x00000004f7400000| Untracked -| 275|0x00000004f7600000, 0x00000004f7600000, 0x00000004f7800000| 0%| F| |TAMS 0x00000004f7600000, 0x00000004f7600000| Untracked -| 276|0x00000004f7800000, 0x00000004f7800000, 0x00000004f7a00000| 0%| F| |TAMS 0x00000004f7800000, 0x00000004f7800000| Untracked -| 277|0x00000004f7a00000, 0x00000004f7a00000, 0x00000004f7c00000| 0%| F| |TAMS 0x00000004f7a00000, 0x00000004f7a00000| Untracked -| 278|0x00000004f7c00000, 0x00000004f7c00000, 0x00000004f7e00000| 0%| F| |TAMS 0x00000004f7c00000, 0x00000004f7c00000| Untracked -| 279|0x00000004f7e00000, 0x00000004f7e00000, 0x00000004f8000000| 0%| F| |TAMS 0x00000004f7e00000, 0x00000004f7e00000| Untracked -| 280|0x00000004f8000000, 0x00000004f8000000, 0x00000004f8200000| 0%| F| |TAMS 0x00000004f8000000, 0x00000004f8000000| Untracked -| 281|0x00000004f8200000, 0x00000004f8200000, 0x00000004f8400000| 0%| F| |TAMS 0x00000004f8200000, 0x00000004f8200000| Untracked -| 282|0x00000004f8400000, 0x00000004f8400000, 0x00000004f8600000| 0%| F| |TAMS 0x00000004f8400000, 0x00000004f8400000| Untracked -| 283|0x00000004f8600000, 0x00000004f8600000, 0x00000004f8800000| 0%| F| |TAMS 0x00000004f8600000, 0x00000004f8600000| Untracked -| 284|0x00000004f8800000, 0x00000004f8800000, 0x00000004f8a00000| 0%| F| |TAMS 0x00000004f8800000, 0x00000004f8800000| Untracked -| 285|0x00000004f8a00000, 0x00000004f8a00000, 0x00000004f8c00000| 0%| F| |TAMS 0x00000004f8a00000, 0x00000004f8a00000| Untracked -| 286|0x00000004f8c00000, 0x00000004f8c00000, 0x00000004f8e00000| 0%| F| |TAMS 0x00000004f8c00000, 0x00000004f8c00000| Untracked -| 287|0x00000004f8e00000, 0x00000004f8e00000, 0x00000004f9000000| 0%| F| |TAMS 0x00000004f8e00000, 0x00000004f8e00000| Untracked -| 288|0x00000004f9000000, 0x00000004f9000000, 0x00000004f9200000| 0%| F| |TAMS 0x00000004f9000000, 0x00000004f9000000| Untracked -| 289|0x00000004f9200000, 0x00000004f9200000, 0x00000004f9400000| 0%| F| |TAMS 0x00000004f9200000, 0x00000004f9200000| Untracked -| 290|0x00000004f9400000, 0x00000004f9400000, 0x00000004f9600000| 0%| F| |TAMS 0x00000004f9400000, 0x00000004f9400000| Untracked -| 291|0x00000004f9600000, 0x00000004f9600000, 0x00000004f9800000| 0%| F| |TAMS 0x00000004f9600000, 0x00000004f9600000| Untracked -| 292|0x00000004f9800000, 0x00000004f9800000, 0x00000004f9a00000| 0%| F| |TAMS 0x00000004f9800000, 0x00000004f9800000| Untracked -| 293|0x00000004f9a00000, 0x00000004f9a00000, 0x00000004f9c00000| 0%| F| |TAMS 0x00000004f9a00000, 0x00000004f9a00000| Untracked -| 294|0x00000004f9c00000, 0x00000004f9c00000, 0x00000004f9e00000| 0%| F| |TAMS 0x00000004f9c00000, 0x00000004f9c00000| Untracked -| 295|0x00000004f9e00000, 0x00000004f9e00000, 0x00000004fa000000| 0%| F| |TAMS 0x00000004f9e00000, 0x00000004f9e00000| Untracked -| 296|0x00000004fa000000, 0x00000004fa000000, 0x00000004fa200000| 0%| F| |TAMS 0x00000004fa000000, 0x00000004fa000000| Untracked -| 297|0x00000004fa200000, 0x00000004fa200000, 0x00000004fa400000| 0%| F| |TAMS 0x00000004fa200000, 0x00000004fa200000| Untracked -| 298|0x00000004fa400000, 0x00000004fa400000, 0x00000004fa600000| 0%| F| |TAMS 0x00000004fa400000, 0x00000004fa400000| Untracked -| 299|0x00000004fa600000, 0x00000004fa600000, 0x00000004fa800000| 0%| F| |TAMS 0x00000004fa600000, 0x00000004fa600000| Untracked -| 300|0x00000004fa800000, 0x00000004fa800000, 0x00000004faa00000| 0%| F| |TAMS 0x00000004fa800000, 0x00000004fa800000| Untracked -| 301|0x00000004faa00000, 0x00000004faa00000, 0x00000004fac00000| 0%| F| |TAMS 0x00000004faa00000, 0x00000004faa00000| Untracked -| 302|0x00000004fac00000, 0x00000004fac00000, 0x00000004fae00000| 0%| F| |TAMS 0x00000004fac00000, 0x00000004fac00000| Untracked -| 303|0x00000004fae00000, 0x00000004fae00000, 0x00000004fb000000| 0%| F| |TAMS 0x00000004fae00000, 0x00000004fae00000| Untracked -| 304|0x00000004fb000000, 0x00000004fb000000, 0x00000004fb200000| 0%| F| |TAMS 0x00000004fb000000, 0x00000004fb000000| Untracked -| 305|0x00000004fb200000, 0x00000004fb200000, 0x00000004fb400000| 0%| F| |TAMS 0x00000004fb200000, 0x00000004fb200000| Untracked -| 306|0x00000004fb400000, 0x00000004fb400000, 0x00000004fb600000| 0%| F| |TAMS 0x00000004fb400000, 0x00000004fb400000| Untracked -| 307|0x00000004fb600000, 0x00000004fb600000, 0x00000004fb800000| 0%| F| |TAMS 0x00000004fb600000, 0x00000004fb600000| Untracked -| 308|0x00000004fb800000, 0x00000004fb800000, 0x00000004fba00000| 0%| F| |TAMS 0x00000004fb800000, 0x00000004fb800000| Untracked -| 309|0x00000004fba00000, 0x00000004fba00000, 0x00000004fbc00000| 0%| F| |TAMS 0x00000004fba00000, 0x00000004fba00000| Untracked -| 310|0x00000004fbc00000, 0x00000004fbc00000, 0x00000004fbe00000| 0%| F| |TAMS 0x00000004fbc00000, 0x00000004fbc00000| Untracked -| 311|0x00000004fbe00000, 0x00000004fbe00000, 0x00000004fc000000| 0%| F| |TAMS 0x00000004fbe00000, 0x00000004fbe00000| Untracked -| 312|0x00000004fc000000, 0x00000004fc000000, 0x00000004fc200000| 0%| F| |TAMS 0x00000004fc000000, 0x00000004fc000000| Untracked -| 313|0x00000004fc200000, 0x00000004fc200000, 0x00000004fc400000| 0%| F| |TAMS 0x00000004fc200000, 0x00000004fc200000| Untracked -| 314|0x00000004fc400000, 0x00000004fc400000, 0x00000004fc600000| 0%| F| |TAMS 0x00000004fc400000, 0x00000004fc400000| Untracked -| 315|0x00000004fc600000, 0x00000004fc600000, 0x00000004fc800000| 0%| F| |TAMS 0x00000004fc600000, 0x00000004fc600000| Untracked -| 316|0x00000004fc800000, 0x00000004fc800000, 0x00000004fca00000| 0%| F| |TAMS 0x00000004fc800000, 0x00000004fc800000| Untracked -| 317|0x00000004fca00000, 0x00000004fca00000, 0x00000004fcc00000| 0%| F| |TAMS 0x00000004fca00000, 0x00000004fca00000| Untracked -| 318|0x00000004fcc00000, 0x00000004fcc00000, 0x00000004fce00000| 0%| F| |TAMS 0x00000004fcc00000, 0x00000004fcc00000| Untracked -| 319|0x00000004fce00000, 0x00000004fce00000, 0x00000004fd000000| 0%| F| |TAMS 0x00000004fce00000, 0x00000004fce00000| Untracked -| 320|0x00000004fd000000, 0x00000004fd000000, 0x00000004fd200000| 0%| F| |TAMS 0x00000004fd000000, 0x00000004fd000000| Untracked -| 321|0x00000004fd200000, 0x00000004fd200000, 0x00000004fd400000| 0%| F| |TAMS 0x00000004fd200000, 0x00000004fd200000| Untracked -| 322|0x00000004fd400000, 0x00000004fd400000, 0x00000004fd600000| 0%| F| |TAMS 0x00000004fd400000, 0x00000004fd400000| Untracked -| 323|0x00000004fd600000, 0x00000004fd600000, 0x00000004fd800000| 0%| F| |TAMS 0x00000004fd600000, 0x00000004fd600000| Untracked -| 324|0x00000004fd800000, 0x00000004fd800000, 0x00000004fda00000| 0%| F| |TAMS 0x00000004fd800000, 0x00000004fd800000| Untracked -| 325|0x00000004fda00000, 0x00000004fda00000, 0x00000004fdc00000| 0%| F| |TAMS 0x00000004fda00000, 0x00000004fda00000| Untracked -| 326|0x00000004fdc00000, 0x00000004fdc00000, 0x00000004fde00000| 0%| F| |TAMS 0x00000004fdc00000, 0x00000004fdc00000| Untracked -| 327|0x00000004fde00000, 0x00000004fde00000, 0x00000004fe000000| 0%| F| |TAMS 0x00000004fde00000, 0x00000004fde00000| Untracked -| 328|0x00000004fe000000, 0x00000004fe000000, 0x00000004fe200000| 0%| F| |TAMS 0x00000004fe000000, 0x00000004fe000000| Untracked -| 329|0x00000004fe200000, 0x00000004fe200000, 0x00000004fe400000| 0%| F| |TAMS 0x00000004fe200000, 0x00000004fe200000| Untracked -| 330|0x00000004fe400000, 0x00000004fe400000, 0x00000004fe600000| 0%| F| |TAMS 0x00000004fe400000, 0x00000004fe400000| Untracked -| 331|0x00000004fe600000, 0x00000004fe600000, 0x00000004fe800000| 0%| F| |TAMS 0x00000004fe600000, 0x00000004fe600000| Untracked -| 332|0x00000004fe800000, 0x00000004fe800000, 0x00000004fea00000| 0%| F| |TAMS 0x00000004fe800000, 0x00000004fe800000| Untracked -| 333|0x00000004fea00000, 0x00000004fea00000, 0x00000004fec00000| 0%| F| |TAMS 0x00000004fea00000, 0x00000004fea00000| Untracked -| 334|0x00000004fec00000, 0x00000004fec00000, 0x00000004fee00000| 0%| F| |TAMS 0x00000004fec00000, 0x00000004fec00000| Untracked -| 335|0x00000004fee00000, 0x00000004fee00000, 0x00000004ff000000| 0%| F| |TAMS 0x00000004fee00000, 0x00000004fee00000| Untracked -| 336|0x00000004ff000000, 0x00000004ff000000, 0x00000004ff200000| 0%| F| |TAMS 0x00000004ff000000, 0x00000004ff000000| Untracked -| 337|0x00000004ff200000, 0x00000004ff200000, 0x00000004ff400000| 0%| F| |TAMS 0x00000004ff200000, 0x00000004ff200000| Untracked -| 338|0x00000004ff400000, 0x00000004ff400000, 0x00000004ff600000| 0%| F| |TAMS 0x00000004ff400000, 0x00000004ff400000| Untracked -| 339|0x00000004ff600000, 0x00000004ff600000, 0x00000004ff800000| 0%| F| |TAMS 0x00000004ff600000, 0x00000004ff600000| Untracked -| 340|0x00000004ff800000, 0x00000004ff800000, 0x00000004ffa00000| 0%| F| |TAMS 0x00000004ff800000, 0x00000004ff800000| Untracked -| 341|0x00000004ffa00000, 0x00000004ffa00000, 0x00000004ffc00000| 0%| F| |TAMS 0x00000004ffa00000, 0x00000004ffa00000| Untracked -| 342|0x00000004ffc00000, 0x00000004ffc00000, 0x00000004ffe00000| 0%| F| |TAMS 0x00000004ffc00000, 0x00000004ffc00000| Untracked -| 343|0x00000004ffe00000, 0x00000004ffe00000, 0x0000000500000000| 0%| F| |TAMS 0x00000004ffe00000, 0x00000004ffe00000| Untracked -| 344|0x0000000500000000, 0x0000000500000000, 0x0000000500200000| 0%| F| |TAMS 0x0000000500000000, 0x0000000500000000| Untracked -| 345|0x0000000500200000, 0x0000000500200000, 0x0000000500400000| 0%| F| |TAMS 0x0000000500200000, 0x0000000500200000| Untracked -| 346|0x0000000500400000, 0x0000000500400000, 0x0000000500600000| 0%| F| |TAMS 0x0000000500400000, 0x0000000500400000| Untracked -| 347|0x0000000500600000, 0x0000000500600000, 0x0000000500800000| 0%| F| |TAMS 0x0000000500600000, 0x0000000500600000| Untracked -| 348|0x0000000500800000, 0x0000000500800000, 0x0000000500a00000| 0%| F| |TAMS 0x0000000500800000, 0x0000000500800000| Untracked -| 349|0x0000000500a00000, 0x0000000500a00000, 0x0000000500c00000| 0%| F| |TAMS 0x0000000500a00000, 0x0000000500a00000| Untracked -| 350|0x0000000500c00000, 0x0000000500c00000, 0x0000000500e00000| 0%| F| |TAMS 0x0000000500c00000, 0x0000000500c00000| Untracked -| 351|0x0000000500e00000, 0x0000000500e00000, 0x0000000501000000| 0%| F| |TAMS 0x0000000500e00000, 0x0000000500e00000| Untracked -| 352|0x0000000501000000, 0x0000000501000000, 0x0000000501200000| 0%| F| |TAMS 0x0000000501000000, 0x0000000501000000| Untracked -| 353|0x0000000501200000, 0x000000050132f438, 0x0000000501400000| 59%| E| |TAMS 0x0000000501200000, 0x0000000501200000| Complete -| 354|0x0000000501400000, 0x0000000501600000, 0x0000000501600000|100%| E|CS|TAMS 0x0000000501400000, 0x0000000501400000| Complete -| 355|0x0000000501600000, 0x0000000501800000, 0x0000000501800000|100%| E|CS|TAMS 0x0000000501600000, 0x0000000501600000| Complete -| 356|0x0000000501800000, 0x0000000501a00000, 0x0000000501a00000|100%| E|CS|TAMS 0x0000000501800000, 0x0000000501800000| Complete -| 357|0x0000000501a00000, 0x0000000501c00000, 0x0000000501c00000|100%| E|CS|TAMS 0x0000000501a00000, 0x0000000501a00000| Complete -| 358|0x0000000501c00000, 0x0000000501e00000, 0x0000000501e00000|100%| E|CS|TAMS 0x0000000501c00000, 0x0000000501c00000| Complete -| 359|0x0000000501e00000, 0x0000000502000000, 0x0000000502000000|100%| E|CS|TAMS 0x0000000501e00000, 0x0000000501e00000| Complete -| 360|0x0000000502000000, 0x0000000502200000, 0x0000000502200000|100%| E|CS|TAMS 0x0000000502000000, 0x0000000502000000| Complete -| 361|0x0000000502200000, 0x0000000502400000, 0x0000000502400000|100%| E|CS|TAMS 0x0000000502200000, 0x0000000502200000| Complete -| 362|0x0000000502400000, 0x0000000502600000, 0x0000000502600000|100%| E|CS|TAMS 0x0000000502400000, 0x0000000502400000| Complete -| 363|0x0000000502600000, 0x0000000502800000, 0x0000000502800000|100%| E|CS|TAMS 0x0000000502600000, 0x0000000502600000| Complete -| 364|0x0000000502800000, 0x0000000502a00000, 0x0000000502a00000|100%| E|CS|TAMS 0x0000000502800000, 0x0000000502800000| Complete -| 365|0x0000000502a00000, 0x0000000502c00000, 0x0000000502c00000|100%| E|CS|TAMS 0x0000000502a00000, 0x0000000502a00000| Complete -| 366|0x0000000502c00000, 0x0000000502e00000, 0x0000000502e00000|100%| E|CS|TAMS 0x0000000502c00000, 0x0000000502c00000| Complete -| 367|0x0000000502e00000, 0x0000000503000000, 0x0000000503000000|100%| E|CS|TAMS 0x0000000502e00000, 0x0000000502e00000| Complete -| 368|0x0000000503000000, 0x0000000503200000, 0x0000000503200000|100%| E|CS|TAMS 0x0000000503000000, 0x0000000503000000| Complete -| 369|0x0000000503200000, 0x0000000503400000, 0x0000000503400000|100%| E|CS|TAMS 0x0000000503200000, 0x0000000503200000| Complete -| 370|0x0000000503400000, 0x0000000503600000, 0x0000000503600000|100%| E|CS|TAMS 0x0000000503400000, 0x0000000503400000| Complete -| 371|0x0000000503600000, 0x0000000503800000, 0x0000000503800000|100%| E|CS|TAMS 0x0000000503600000, 0x0000000503600000| Complete -| 372|0x0000000503800000, 0x0000000503a00000, 0x0000000503a00000|100%| E|CS|TAMS 0x0000000503800000, 0x0000000503800000| Complete -| 373|0x0000000503a00000, 0x0000000503c00000, 0x0000000503c00000|100%| E|CS|TAMS 0x0000000503a00000, 0x0000000503a00000| Complete -| 374|0x0000000503c00000, 0x0000000503e00000, 0x0000000503e00000|100%| E|CS|TAMS 0x0000000503c00000, 0x0000000503c00000| Complete -| 375|0x0000000503e00000, 0x0000000504000000, 0x0000000504000000|100%| E|CS|TAMS 0x0000000503e00000, 0x0000000503e00000| Complete -| 376|0x0000000504000000, 0x0000000504200000, 0x0000000504200000|100%| E|CS|TAMS 0x0000000504000000, 0x0000000504000000| Complete -| 377|0x0000000504200000, 0x0000000504400000, 0x0000000504400000|100%| E|CS|TAMS 0x0000000504200000, 0x0000000504200000| Complete -| 378|0x0000000504400000, 0x0000000504600000, 0x0000000504600000|100%| E|CS|TAMS 0x0000000504400000, 0x0000000504400000| Complete -| 379|0x0000000504600000, 0x0000000504800000, 0x0000000504800000|100%| E|CS|TAMS 0x0000000504600000, 0x0000000504600000| Complete -| 380|0x0000000504800000, 0x0000000504a00000, 0x0000000504a00000|100%| E|CS|TAMS 0x0000000504800000, 0x0000000504800000| Complete -| 381|0x0000000504a00000, 0x0000000504c00000, 0x0000000504c00000|100%| E|CS|TAMS 0x0000000504a00000, 0x0000000504a00000| Complete -| 382|0x0000000504c00000, 0x0000000504e00000, 0x0000000504e00000|100%| E|CS|TAMS 0x0000000504c00000, 0x0000000504c00000| Complete -| 383|0x0000000504e00000, 0x0000000505000000, 0x0000000505000000|100%| E|CS|TAMS 0x0000000504e00000, 0x0000000504e00000| Complete -| 384|0x0000000505000000, 0x0000000505200000, 0x0000000505200000|100%| E|CS|TAMS 0x0000000505000000, 0x0000000505000000| Complete -| 385|0x0000000505200000, 0x0000000505400000, 0x0000000505400000|100%| E|CS|TAMS 0x0000000505200000, 0x0000000505200000| Complete -| 386|0x0000000505400000, 0x0000000505600000, 0x0000000505600000|100%| E|CS|TAMS 0x0000000505400000, 0x0000000505400000| Complete -| 387|0x0000000505600000, 0x0000000505800000, 0x0000000505800000|100%| E|CS|TAMS 0x0000000505600000, 0x0000000505600000| Complete -| 388|0x0000000505800000, 0x0000000505a00000, 0x0000000505a00000|100%| E|CS|TAMS 0x0000000505800000, 0x0000000505800000| Complete -| 389|0x0000000505a00000, 0x0000000505c00000, 0x0000000505c00000|100%| E|CS|TAMS 0x0000000505a00000, 0x0000000505a00000| Complete -| 390|0x0000000505c00000, 0x0000000505e00000, 0x0000000505e00000|100%| E|CS|TAMS 0x0000000505c00000, 0x0000000505c00000| Complete -| 391|0x0000000505e00000, 0x0000000506000000, 0x0000000506000000|100%| E|CS|TAMS 0x0000000505e00000, 0x0000000505e00000| Complete -| 392|0x0000000506000000, 0x0000000506200000, 0x0000000506200000|100%| E|CS|TAMS 0x0000000506000000, 0x0000000506000000| Complete -| 393|0x0000000506200000, 0x0000000506400000, 0x0000000506400000|100%| E|CS|TAMS 0x0000000506200000, 0x0000000506200000| Complete -| 394|0x0000000506400000, 0x0000000506600000, 0x0000000506600000|100%| E|CS|TAMS 0x0000000506400000, 0x0000000506400000| Complete -| 395|0x0000000506600000, 0x0000000506800000, 0x0000000506800000|100%| E|CS|TAMS 0x0000000506600000, 0x0000000506600000| Complete -| 396|0x0000000506800000, 0x0000000506a00000, 0x0000000506a00000|100%| E|CS|TAMS 0x0000000506800000, 0x0000000506800000| Complete -| 397|0x0000000506a00000, 0x0000000506c00000, 0x0000000506c00000|100%| E|CS|TAMS 0x0000000506a00000, 0x0000000506a00000| Complete -| 398|0x0000000506c00000, 0x0000000506e00000, 0x0000000506e00000|100%| E|CS|TAMS 0x0000000506c00000, 0x0000000506c00000| Complete -| 399|0x0000000506e00000, 0x0000000507000000, 0x0000000507000000|100%| E|CS|TAMS 0x0000000506e00000, 0x0000000506e00000| Complete -| 400|0x0000000507000000, 0x0000000507200000, 0x0000000507200000|100%| E|CS|TAMS 0x0000000507000000, 0x0000000507000000| Complete -| 401|0x0000000507200000, 0x0000000507400000, 0x0000000507400000|100%| E|CS|TAMS 0x0000000507200000, 0x0000000507200000| Complete -| 402|0x0000000507400000, 0x0000000507600000, 0x0000000507600000|100%| E|CS|TAMS 0x0000000507400000, 0x0000000507400000| Complete -| 403|0x0000000507600000, 0x0000000507800000, 0x0000000507800000|100%| E|CS|TAMS 0x0000000507600000, 0x0000000507600000| Complete -| 404|0x0000000507800000, 0x0000000507a00000, 0x0000000507a00000|100%| E|CS|TAMS 0x0000000507800000, 0x0000000507800000| Complete -| 405|0x0000000507a00000, 0x0000000507c00000, 0x0000000507c00000|100%| E|CS|TAMS 0x0000000507a00000, 0x0000000507a00000| Complete -| 406|0x0000000507c00000, 0x0000000507e00000, 0x0000000507e00000|100%| E|CS|TAMS 0x0000000507c00000, 0x0000000507c00000| Complete -| 407|0x0000000507e00000, 0x0000000508000000, 0x0000000508000000|100%| E|CS|TAMS 0x0000000507e00000, 0x0000000507e00000| Complete -| 408|0x0000000508000000, 0x0000000508200000, 0x0000000508200000|100%| E|CS|TAMS 0x0000000508000000, 0x0000000508000000| Complete -| 409|0x0000000508200000, 0x0000000508400000, 0x0000000508400000|100%| E|CS|TAMS 0x0000000508200000, 0x0000000508200000| Complete -| 410|0x0000000508400000, 0x0000000508600000, 0x0000000508600000|100%| E|CS|TAMS 0x0000000508400000, 0x0000000508400000| Complete -| 411|0x0000000508600000, 0x0000000508800000, 0x0000000508800000|100%| E|CS|TAMS 0x0000000508600000, 0x0000000508600000| Complete -| 412|0x0000000508800000, 0x0000000508a00000, 0x0000000508a00000|100%| E|CS|TAMS 0x0000000508800000, 0x0000000508800000| Complete -| 413|0x0000000508a00000, 0x0000000508c00000, 0x0000000508c00000|100%| E|CS|TAMS 0x0000000508a00000, 0x0000000508a00000| Complete -| 414|0x0000000508c00000, 0x0000000508e00000, 0x0000000508e00000|100%| E|CS|TAMS 0x0000000508c00000, 0x0000000508c00000| Complete -| 415|0x0000000508e00000, 0x0000000509000000, 0x0000000509000000|100%| E|CS|TAMS 0x0000000508e00000, 0x0000000508e00000| Complete -| 416|0x0000000509000000, 0x0000000509200000, 0x0000000509200000|100%| E|CS|TAMS 0x0000000509000000, 0x0000000509000000| Complete -| 417|0x0000000509200000, 0x0000000509400000, 0x0000000509400000|100%| E|CS|TAMS 0x0000000509200000, 0x0000000509200000| Complete -| 418|0x0000000509400000, 0x0000000509600000, 0x0000000509600000|100%| E|CS|TAMS 0x0000000509400000, 0x0000000509400000| Complete -| 419|0x0000000509600000, 0x0000000509800000, 0x0000000509800000|100%| E|CS|TAMS 0x0000000509600000, 0x0000000509600000| Complete -| 420|0x0000000509800000, 0x0000000509a00000, 0x0000000509a00000|100%| E|CS|TAMS 0x0000000509800000, 0x0000000509800000| Complete -| 421|0x0000000509a00000, 0x0000000509c00000, 0x0000000509c00000|100%| E|CS|TAMS 0x0000000509a00000, 0x0000000509a00000| Complete -| 422|0x0000000509c00000, 0x0000000509e00000, 0x0000000509e00000|100%| E|CS|TAMS 0x0000000509c00000, 0x0000000509c00000| Complete -| 423|0x0000000509e00000, 0x000000050a000000, 0x000000050a000000|100%| E|CS|TAMS 0x0000000509e00000, 0x0000000509e00000| Complete -| 424|0x000000050a000000, 0x000000050a200000, 0x000000050a200000|100%| E|CS|TAMS 0x000000050a000000, 0x000000050a000000| Complete -| 425|0x000000050a200000, 0x000000050a400000, 0x000000050a400000|100%| E|CS|TAMS 0x000000050a200000, 0x000000050a200000| Complete -| 426|0x000000050a400000, 0x000000050a600000, 0x000000050a600000|100%| E|CS|TAMS 0x000000050a400000, 0x000000050a400000| Complete -| 427|0x000000050a600000, 0x000000050a800000, 0x000000050a800000|100%| E|CS|TAMS 0x000000050a600000, 0x000000050a600000| Complete -| 428|0x000000050a800000, 0x000000050aa00000, 0x000000050aa00000|100%| E|CS|TAMS 0x000000050a800000, 0x000000050a800000| Complete -| 429|0x000000050aa00000, 0x000000050ac00000, 0x000000050ac00000|100%| E|CS|TAMS 0x000000050aa00000, 0x000000050aa00000| Complete -| 430|0x000000050ac00000, 0x000000050ae00000, 0x000000050ae00000|100%| E|CS|TAMS 0x000000050ac00000, 0x000000050ac00000| Complete -| 431|0x000000050ae00000, 0x000000050b000000, 0x000000050b000000|100%| E|CS|TAMS 0x000000050ae00000, 0x000000050ae00000| Complete -| 432|0x000000050b000000, 0x000000050b200000, 0x000000050b200000|100%| E|CS|TAMS 0x000000050b000000, 0x000000050b000000| Complete -| 433|0x000000050b200000, 0x000000050b400000, 0x000000050b400000|100%| E|CS|TAMS 0x000000050b200000, 0x000000050b200000| Complete -| 434|0x000000050b400000, 0x000000050b600000, 0x000000050b600000|100%| E|CS|TAMS 0x000000050b400000, 0x000000050b400000| Complete -| 435|0x000000050b600000, 0x000000050b800000, 0x000000050b800000|100%| E|CS|TAMS 0x000000050b600000, 0x000000050b600000| Complete -| 436|0x000000050b800000, 0x000000050ba00000, 0x000000050ba00000|100%| E|CS|TAMS 0x000000050b800000, 0x000000050b800000| Complete -| 437|0x000000050ba00000, 0x000000050bc00000, 0x000000050bc00000|100%| E|CS|TAMS 0x000000050ba00000, 0x000000050ba00000| Complete -| 438|0x000000050bc00000, 0x000000050be00000, 0x000000050be00000|100%| E|CS|TAMS 0x000000050bc00000, 0x000000050bc00000| Complete -| 439|0x000000050be00000, 0x000000050c000000, 0x000000050c000000|100%| E|CS|TAMS 0x000000050be00000, 0x000000050be00000| Complete -| 440|0x000000050c000000, 0x000000050c200000, 0x000000050c200000|100%| E|CS|TAMS 0x000000050c000000, 0x000000050c000000| Complete -| 441|0x000000050c200000, 0x000000050c400000, 0x000000050c400000|100%| E|CS|TAMS 0x000000050c200000, 0x000000050c200000| Complete -| 442|0x000000050c400000, 0x000000050c600000, 0x000000050c600000|100%| E|CS|TAMS 0x000000050c400000, 0x000000050c400000| Complete -| 443|0x000000050c600000, 0x000000050c800000, 0x000000050c800000|100%| E|CS|TAMS 0x000000050c600000, 0x000000050c600000| Complete -| 444|0x000000050c800000, 0x000000050ca00000, 0x000000050ca00000|100%| E|CS|TAMS 0x000000050c800000, 0x000000050c800000| Complete -| 445|0x000000050ca00000, 0x000000050cc00000, 0x000000050cc00000|100%| E|CS|TAMS 0x000000050ca00000, 0x000000050ca00000| Complete -| 446|0x000000050cc00000, 0x000000050ce00000, 0x000000050ce00000|100%| E|CS|TAMS 0x000000050cc00000, 0x000000050cc00000| Complete -| 447|0x000000050ce00000, 0x000000050d000000, 0x000000050d000000|100%| E|CS|TAMS 0x000000050ce00000, 0x000000050ce00000| Complete -| 448|0x000000050d000000, 0x000000050d200000, 0x000000050d200000|100%| E|CS|TAMS 0x000000050d000000, 0x000000050d000000| Complete -| 449|0x000000050d200000, 0x000000050d400000, 0x000000050d400000|100%| E|CS|TAMS 0x000000050d200000, 0x000000050d200000| Complete -| 450|0x000000050d400000, 0x000000050d600000, 0x000000050d600000|100%| E|CS|TAMS 0x000000050d400000, 0x000000050d400000| Complete -| 451|0x000000050d600000, 0x000000050d800000, 0x000000050d800000|100%| E|CS|TAMS 0x000000050d600000, 0x000000050d600000| Complete -| 452|0x000000050d800000, 0x000000050da00000, 0x000000050da00000|100%| E|CS|TAMS 0x000000050d800000, 0x000000050d800000| Complete -| 453|0x000000050da00000, 0x000000050dc00000, 0x000000050dc00000|100%| E|CS|TAMS 0x000000050da00000, 0x000000050da00000| Complete -| 454|0x000000050dc00000, 0x000000050de00000, 0x000000050de00000|100%| E|CS|TAMS 0x000000050dc00000, 0x000000050dc00000| Complete -| 455|0x000000050de00000, 0x000000050e000000, 0x000000050e000000|100%| E|CS|TAMS 0x000000050de00000, 0x000000050de00000| Complete -| 456|0x000000050e000000, 0x000000050e200000, 0x000000050e200000|100%| E|CS|TAMS 0x000000050e000000, 0x000000050e000000| Complete -| 457|0x000000050e200000, 0x000000050e400000, 0x000000050e400000|100%| E|CS|TAMS 0x000000050e200000, 0x000000050e200000| Complete -| 458|0x000000050e400000, 0x000000050e600000, 0x000000050e600000|100%| E|CS|TAMS 0x000000050e400000, 0x000000050e400000| Complete -| 459|0x000000050e600000, 0x000000050e800000, 0x000000050e800000|100%| E|CS|TAMS 0x000000050e600000, 0x000000050e600000| Complete -| 460|0x000000050e800000, 0x000000050ea00000, 0x000000050ea00000|100%| E|CS|TAMS 0x000000050e800000, 0x000000050e800000| Complete -| 461|0x000000050ea00000, 0x000000050ec00000, 0x000000050ec00000|100%| E|CS|TAMS 0x000000050ea00000, 0x000000050ea00000| Complete -| 462|0x000000050ec00000, 0x000000050ee00000, 0x000000050ee00000|100%| E|CS|TAMS 0x000000050ec00000, 0x000000050ec00000| Complete -| 463|0x000000050ee00000, 0x000000050f000000, 0x000000050f000000|100%| E|CS|TAMS 0x000000050ee00000, 0x000000050ee00000| Complete -| 464|0x000000050f000000, 0x000000050f200000, 0x000000050f200000|100%| E|CS|TAMS 0x000000050f000000, 0x000000050f000000| Complete -| 767|0x0000000534e00000, 0x0000000535000000, 0x0000000535000000|100%| E|CS|TAMS 0x0000000534e00000, 0x0000000534e00000| Complete -|2046|0x00000005d4c00000, 0x00000005d4d74000, 0x00000005d4e00000| 72%|OA| |TAMS 0x00000005d4d74000, 0x00000005d4c00000| Untracked -|2047|0x00000005d4e00000, 0x00000005d4e74000, 0x00000005d5000000| 22%|CA| |TAMS 0x00000005d4e74000, 0x00000005d4e00000| Untracked - -Card table byte_map: [0x000000011089c000,0x000000011109c000] _byte_map_base: 0x000000010e1f4000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x0000000afccc4510, (CMBitMap*) 0x0000000afccc4550 - Prev Bits: [0x000000011189c000, 0x000000011589c000) - Next Bits: [0x000000011589c000, 0x000000011989c000) - -Polling page: 0x00000001021a4000 - -Metaspace: - -Usage: - Non-class: 129.98 MB used. - Class: 17.71 MB used. - Both: 147.69 MB used. - -Virtual space: - Non-class space: 192.00 MB reserved, 130.56 MB ( 68%) committed, 3 nodes. - Class space: 1.00 GB reserved, 18.19 MB ( 2%) committed, 1 nodes. - Both: 1.19 GB reserved, 148.75 MB ( 12%) committed. - -Chunk freelists: - Non-Class: 12.84 MB - Class: 13.85 MB - Both: 26.69 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 217.12 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 12. -num_arena_births: 2134. -num_arena_deaths: 412. -num_vsnodes_births: 4. -num_vsnodes_deaths: 0. -num_space_committed: 2377. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 582. -num_chunks_taken_from_freelist: 8069. -num_chunk_merges: 221. -num_chunk_splits: 5784. -num_chunks_enlarged: 4367. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=119168Kb used=10598Kb max_used=10598Kb free=108569Kb - bounds [0x00000001249a4000, 0x0000000125404000, 0x000000012be04000] -CodeHeap 'profiled nmethods': size=119152Kb used=21762Kb max_used=21762Kb free=97389Kb - bounds [0x000000011ce04000, 0x000000011e354000, 0x0000000124260000] -CodeHeap 'non-nmethods': size=7440Kb used=3320Kb max_used=3502Kb free=4119Kb - bounds [0x0000000124260000, 0x00000001245e0000, 0x00000001249a4000] - total_blobs=13178 nmethods=12116 adapters=973 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 21.310 Thread 0x0000000afb471200 16525 3 scala.collection.mutable.IndexedSeq::companion (13 bytes) -Event: 21.310 Thread 0x0000000af816aa00 16526 3 java.util.Collections::synchronizedSet (10 bytes) -Event: 21.310 Thread 0x0000000afb471200 nmethod 16525 0x000000011e343a10 code [0x000000011e343bc0, 0x000000011e343d48] -Event: 21.310 Thread 0x0000000afb471200 16527 3 org.apache.spark.util.Utils$::bytesToString (146 bytes) -Event: 21.310 Thread 0x0000000afcd82400 nmethod 16523 0x000000011e343e90 code [0x000000011e344080, 0x000000011e344558] -Event: 21.310 Thread 0x0000000afcd82400 16529 2 scala.math.Numeric::zero$ (13 bytes) -Event: 21.310 Thread 0x0000000afb470c00 nmethod 16524 0x000000011e344790 code [0x000000011e3449c0, 0x000000011e344fb8] -Event: 21.310 Thread 0x0000000afcd82400 nmethod 16529 0x000000011e345210 code [0x000000011e3453c0, 0x000000011e345578] -Event: 21.310 Thread 0x0000000afcd82400 16528 3 java.util.regex.Pattern::createGroup (86 bytes) -Event: 21.310 Thread 0x0000000afb470c00 16530 3 org.apache.logging.log4j.core.impl.JdkMapAdapterStringMap:: (35 bytes) -Event: 21.310 Thread 0x0000000af816aa00 nmethod 16526 0x000000011e345710 code [0x000000011e345940, 0x000000011e345fb8] -Event: 21.310 Thread 0x0000000af816aa00 16532 3 java.util.regex.Pattern::accept (43 bytes) -Event: 21.311 Thread 0x0000000afb470c00 nmethod 16530 0x000000011e346210 code [0x000000011e346400, 0x000000011e3468f8] -Event: 21.311 Thread 0x0000000afb470c00 16531 3 org.apache.logging.log4j.core.impl.JdkMapAdapterStringMap::freeze (15 bytes) -Event: 21.311 Thread 0x0000000af816aa00 nmethod 16532 0x000000011e346b10 code [0x000000011e346d00, 0x000000011e347038] -Event: 21.311 Thread 0x0000000af816aa00 16533 3 org.apache.logging.log4j.core.impl.ThreadContextDataInjector$ForDefaultThreadContextMap::frozenStringMap (29 bytes) -Event: 21.311 Thread 0x0000000afcd82400 nmethod 16528 0x000000011e347190 code [0x000000011e3473c0, 0x000000011e347b58] -Event: 21.311 Thread 0x0000000afb471200 nmethod 16527 0x000000011e347e10 code [0x000000011e3480c0, 0x000000011e348b38] -Event: 21.311 Thread 0x0000000afb470c00 nmethod 16531 0x000000011e348f10 code [0x000000011e3490c0, 0x000000011e349248] -Event: 21.311 Thread 0x0000000af816aa00 nmethod 16533 0x000000011e349390 code [0x000000011e349580, 0x000000011e3497e8] - -GC Heap History (20 events): -Event: 6.465 GC heap before -{Heap before GC invocations=27 (full 0): - garbage-first heap total 342016K, used 303817K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 96 young (196608K), 1 survivors (2048K) - Metaspace used 118486K, committed 119232K, reserved 1179648K - class space used 14224K, committed 14592K, reserved 1048576K -} -Event: 6.468 GC heap after -{Heap after GC invocations=28 (full 0): - garbage-first heap total 342016K, used 125195K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 11 young (22528K), 11 survivors (22528K) - Metaspace used 118486K, committed 119232K, reserved 1179648K - class space used 14224K, committed 14592K, reserved 1048576K -} -Event: 6.602 GC heap before -{Heap before GC invocations=28 (full 0): - garbage-first heap total 342016K, used 209163K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 54 young (110592K), 11 survivors (22528K) - Metaspace used 120736K, committed 121600K, reserved 1179648K - class space used 14577K, committed 14976K, reserved 1048576K -} -Event: 6.606 GC heap after -{Heap after GC invocations=29 (full 0): - garbage-first heap total 342016K, used 133061K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 120736K, committed 121600K, reserved 1179648K - class space used 14577K, committed 14976K, reserved 1048576K -} -Event: 6.855 GC heap before -{Heap before GC invocations=30 (full 0): - garbage-first heap total 342016K, used 276421K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 70 young (143360K), 4 survivors (8192K) - Metaspace used 124828K, committed 125696K, reserved 1179648K - class space used 14919K, committed 15360K, reserved 1048576K -} -Event: 6.856 GC heap after -{Heap after GC invocations=31 (full 0): - garbage-first heap total 342016K, used 147033K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 6 young (12288K), 6 survivors (12288K) - Metaspace used 124828K, committed 125696K, reserved 1179648K - class space used 14919K, committed 15360K, reserved 1048576K -} -Event: 7.137 GC heap before -{Heap before GC invocations=31 (full 0): - garbage-first heap total 342016K, used 274009K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 69 young (141312K), 6 survivors (12288K) - Metaspace used 129032K, committed 129920K, reserved 1179648K - class space used 15269K, committed 15680K, reserved 1048576K -} -Event: 7.139 GC heap after -{Heap after GC invocations=32 (full 0): - garbage-first heap total 342016K, used 148364K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 4 young (8192K), 4 survivors (8192K) - Metaspace used 129032K, committed 129920K, reserved 1179648K - class space used 15269K, committed 15680K, reserved 1048576K -} -Event: 7.199 GC heap before -{Heap before GC invocations=33 (full 0): - garbage-first heap total 342016K, used 195468K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 28 young (57344K), 4 survivors (8192K) - Metaspace used 130550K, committed 131328K, reserved 1179648K - class space used 15456K, committed 15808K, reserved 1048576K -} -Event: 7.201 GC heap after -{Heap after GC invocations=34 (full 0): - garbage-first heap total 342016K, used 148042K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 130550K, committed 131328K, reserved 1179648K - class space used 15456K, committed 15808K, reserved 1048576K -} -Event: 7.326 GC heap before -{Heap before GC invocations=35 (full 0): - garbage-first heap total 342016K, used 201290K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 26 young (53248K), 3 survivors (6144K) - Metaspace used 132612K, committed 133376K, reserved 1179648K - class space used 15837K, committed 16192K, reserved 1048576K -} -Event: 7.328 GC heap after -{Heap after GC invocations=36 (full 0): - garbage-first heap total 342016K, used 144714K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 132612K, committed 133376K, reserved 1179648K - class space used 15837K, committed 16192K, reserved 1048576K -} -Event: 7.565 GC heap before -{Heap before GC invocations=37 (full 0): - garbage-first heap total 342016K, used 273738K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 69 young (141312K), 2 survivors (4096K) - Metaspace used 139445K, committed 140224K, reserved 1179648K - class space used 16694K, committed 17024K, reserved 1048576K -} -Event: 7.567 GC heap after -{Heap after GC invocations=38 (full 0): - garbage-first heap total 342016K, used 125903K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 139445K, committed 140224K, reserved 1179648K - class space used 16694K, committed 17024K, reserved 1048576K -} -Event: 7.580 GC heap before -{Heap before GC invocations=38 (full 0): - garbage-first heap total 342016K, used 136143K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 8 young (16384K), 3 survivors (6144K) - Metaspace used 140017K, committed 140736K, reserved 1179648K - class space used 16762K, committed 17088K, reserved 1048576K -} -Event: 7.582 GC heap after -{Heap after GC invocations=39 (full 0): - garbage-first heap total 342016K, used 124142K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 1 young (2048K), 1 survivors (2048K) - Metaspace used 140017K, committed 140736K, reserved 1179648K - class space used 16762K, committed 17088K, reserved 1048576K -} -Event: 7.851 GC heap before -{Heap before GC invocations=39 (full 0): - garbage-first heap total 342016K, used 287982K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 78 young (159744K), 1 survivors (2048K) - Metaspace used 148180K, committed 149120K, reserved 1245184K - class space used 17685K, committed 18112K, reserved 1048576K -} -Event: 7.853 GC heap after -{Heap after GC invocations=40 (full 0): - garbage-first heap total 958464K, used 126026K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 148180K, committed 149120K, reserved 1245184K - class space used 17685K, committed 18112K, reserved 1048576K -} -Event: 9.193 GC heap before -{Heap before GC invocations=40 (full 0): - garbage-first heap total 958464K, used 695370K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 280 young (573440K), 2 survivors (4096K) - Metaspace used 148181K, committed 149120K, reserved 1245184K - class space used 17686K, committed 18112K, reserved 1048576K -} -Event: 9.194 GC heap after -{Heap after GC invocations=41 (full 0): - garbage-first heap total 958464K, used 126097K [0x00000004d5000000, 0x00000005d5000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 148181K, committed 149120K, reserved 1245184K - class space used 17686K, committed 18112K, reserved 1048576K -} - -Dll operation events (20 events): -Event: 3.646 Loading shared library /usr/lib/java/libcomet.jnilib failed, dlopen(/usr/lib/java/libcomet.jnilib, 0x0001): tried: '/usr/lib/java/libcomet.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/libcomet.jnilib' (no such file), '/usr/li -Event: 3.646 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.dylib' (no such file), -Event: 3.646 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/libcomet.jnilib' (no such fil -Event: 5.628 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib -Event: 6.130 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.dylib, 0x0001): tried: '/Library/Java/JavaVirtualMach -Event: 6.130 Loading shared library /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib failed, dlopen(/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/JavaVirtualMa -Event: 6.130 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Pre -Event: 6.130 Loading shared library /Users/andy/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Users/andy/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/ -Event: 6.130 Loading shared library /Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java/Ex -Event: 6.130 Loading shared library /Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/Library/Java -Event: 6.130 Loading shared library /Network/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 6.130 Loading shared library /Network/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/Network/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/Network/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/C -Event: 6.130 Loading shared library /System/Library/Java/Extensions/liblz4-java.dylib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.dylib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptex -Event: 6.130 Loading shared library /System/Library/Java/Extensions/liblz4-java.jnilib failed, dlopen(/System/Library/Java/Extensions/liblz4-java.jnilib, 0x0001): tried: '/System/Library/Java/Extensions/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryp -Event: 6.130 Loading shared library /usr/lib/java/liblz4-java.dylib failed, dlopen(/usr/lib/java/liblz4-java.dylib, 0x0001): tried: '/usr/lib/java/liblz4-java.dylib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.dylib' (no such file), -Event: 6.131 Loading shared library /usr/lib/java/liblz4-java.jnilib failed, dlopen(/usr/lib/java/liblz4-java.jnilib, 0x0001): tried: '/usr/lib/java/liblz4-java.jnilib' (no such file), '/System/Volumes/Preboot/Cryptexes/OS/usr/lib/java/liblz4-java.jnilib' (no such fil -Event: 6.131 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.dylib' (no su -Event: 6.131 Loading shared library /Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib failed, dlopen(/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib, 0x0001): tried: '/Users/andy/git/apache/datafusion-comet/spark/liblz4-java.jnilib' (no -Event: 6.278 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15423878368099207039.dylib -Event: 7.020 Loaded shared library /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-afedc8a6-4493-4c3c-aed3-c280414ad816-libsnappyjava.dylib - -Deoptimization events (20 events): -Event: 21.297 Thread 0x0000000afb041800 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c -Event: 21.297 Thread 0x0000000afb041800 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 -Event: 21.297 Thread 0x0000000afb041800 DEOPT PACKING pc=0x000000012537452c sp=0x000000017ec661c0 -Event: 21.297 Thread 0x0000000afb041800 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017ec66160 mode 2 -Event: 21.297 Thread 0x0000000afb042400 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c -Event: 21.297 Thread 0x0000000afb042400 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 -Event: 21.297 Thread 0x0000000afb042400 DEOPT PACKING pc=0x000000012537452c sp=0x000000017f47e1c0 -Event: 21.297 Thread 0x0000000afb042400 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f47e160 mode 2 -Event: 21.297 Thread 0x0000000af816a400 Uncommon trap: trap_request=0xffffff66 fr.pc=0x000000012538bb20 relative=0x00000000000024e0 -Event: 21.297 Thread 0x0000000af816a400 Uncommon trap: reason=speculate_class_check action=maybe_recompile pc=0x000000012538bb20 method=java.io.ObjectInputStream.readOrdinaryObject(Z)Ljava/lang/Object; @ 226 c2 -Event: 21.297 Thread 0x0000000af816a400 DEOPT PACKING pc=0x000000012538bb20 sp=0x000000017fea2260 -Event: 21.297 Thread 0x0000000af816a400 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017fea2220 mode 2 -Event: 21.298 Thread 0x0000000afb041e00 Uncommon trap: trap_request=0xffffff66 fr.pc=0x000000012538bb20 relative=0x00000000000024e0 -Event: 21.298 Thread 0x0000000afb041e00 Uncommon trap: reason=speculate_class_check action=maybe_recompile pc=0x000000012538bb20 method=java.io.ObjectInputStream.readOrdinaryObject(Z)Ljava/lang/Object; @ 226 c2 -Event: 21.298 Thread 0x0000000afb041e00 DEOPT PACKING pc=0x000000012538bb20 sp=0x000000017f06ffd0 -Event: 21.298 Thread 0x0000000afb041e00 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f06ff90 mode 2 -Event: 21.298 Thread 0x0000000afb042a00 Uncommon trap: trap_request=0xffffffd6 fr.pc=0x000000012537452c relative=0x000000000000036c -Event: 21.298 Thread 0x0000000afb042a00 Uncommon trap: reason=array_check action=maybe_recompile pc=0x000000012537452c method=java.util.Arrays.fill([Ljava/lang/Object;Ljava/lang/Object;)V @ 13 c2 -Event: 21.298 Thread 0x0000000afb042a00 DEOPT PACKING pc=0x000000012537452c sp=0x000000017f88a1c0 -Event: 21.298 Thread 0x0000000afb042a00 DEOPT UNPACKING pc=0x00000001242a791c sp=0x000000017f88a160 mode 2 - -Classes loaded (20 events): -Event: 7.732 Loading class java/lang/Class$EnclosingMethodInfo -Event: 7.732 Loading class java/lang/Class$EnclosingMethodInfo done -Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper -Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$Unmapper -Event: 16.521 Loading class jdk/internal/access/foreign/UnmapperProxy -Event: 16.521 Loading class jdk/internal/access/foreign/UnmapperProxy done -Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$Unmapper done -Event: 16.521 Loading class sun/nio/ch/FileChannelImpl$DefaultUnmapper done -Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask -Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask -Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$BulkTask done -Event: 21.094 Loading class java/util/concurrent/ConcurrentHashMap$ForEachValueTask done -Event: 21.095 Loading class java/util/function/LongBinaryOperator -Event: 21.095 Loading class java/util/function/LongBinaryOperator done -Event: 21.107 Loading class sun/nio/ch/ChannelInputStream -Event: 21.107 Loading class sun/nio/ch/ChannelInputStream done -Event: 21.179 Loading class java/math/BigDecimal$StringBuilderHelper -Event: 21.179 Loading class java/math/BigDecimal$StringBuilderHelper done -Event: 21.241 Loading class java/nio/channels/Channels$ReadableByteChannelImpl -Event: 21.241 Loading class java/nio/channels/Channels$ReadableByteChannelImpl done - -Classes unloaded (20 events): -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3c000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3c000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b800 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b800' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b400 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b400' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3ac00 'java/lang/invoke/LambdaForm$MH+0x000000d001e3ac00' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3a800 'java/lang/invoke/LambdaForm$MH+0x000000d001e3a800' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3b000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3b000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e3a000 'java/lang/invoke/LambdaForm$MH+0x000000d001e3a000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e39c00' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39800 'java/lang/invoke/LambdaForm$MH+0x000000d001e39800' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e39000 'java/lang/invoke/LambdaForm$MH+0x000000d001e39000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e38c00' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38800 'java/lang/invoke/LambdaForm$MH+0x000000d001e38800' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e38000 'java/lang/invoke/LambdaForm$MH+0x000000d001e38000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e37c00' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37400 'java/lang/invoke/LambdaForm$MH+0x000000d001e37400' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37800 'java/lang/invoke/LambdaForm$MH+0x000000d001e37800' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e37000 'java/lang/invoke/LambdaForm$MH+0x000000d001e37000' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36400 'java/lang/invoke/LambdaForm$MH+0x000000d001e36400' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36c00 'java/lang/invoke/LambdaForm$MH+0x000000d001e36c00' -Event: 7.211 Thread 0x0000000102971070 Unloading class 0x000000d001e36800 'java/lang/invoke/LambdaForm$MH+0x000000d001e36800' - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 21.237 Thread 0x0000000afb042400 Exception (0x0000000502c9d4e0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.237 Thread 0x0000000afb041e00 Exception (0x0000000502a3b288) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.237 Thread 0x0000000afb042a00 Exception (0x0000000502b2d308) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.237 Thread 0x0000000afb041800 Exception (0x0000000502de8160) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.237 Thread 0x0000000afb041200 Exception (0x0000000503390408) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b41728) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033dd288) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502dfc750) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb20d8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041e00 Exception (0x0000000502846808) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b455b8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502900ea0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033e1238) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb5ec8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041e00 Exception (0x000000050284a5e0) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042a00 Exception (0x0000000502b48e98) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041800 Exception (0x0000000502904730) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041200 Exception (0x00000005033e4ac8) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb041e00 Exception (0x000000050284de70) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] -Event: 21.239 Thread 0x0000000afb042400 Exception (0x0000000502cb9758) -thrown [src/hotspot/share/interpreter/linkResolver.cpp, line 759] - -VM Operations (20 events): -Event: 7.989 Executing VM operation: HandshakeAllThreads -Event: 7.989 Executing VM operation: HandshakeAllThreads done -Event: 7.995 Executing VM operation: ICBufferFull -Event: 7.995 Executing VM operation: ICBufferFull done -Event: 8.996 Executing VM operation: Cleanup -Event: 8.996 Executing VM operation: Cleanup done -Event: 9.193 Executing VM operation: G1CollectForAllocation -Event: 9.194 Executing VM operation: G1CollectForAllocation done -Event: 12.209 Executing VM operation: Cleanup -Event: 12.209 Executing VM operation: Cleanup done -Event: 17.234 Executing VM operation: Cleanup -Event: 17.234 Executing VM operation: Cleanup done -Event: 21.127 Executing VM operation: ICBufferFull -Event: 21.127 Executing VM operation: ICBufferFull done -Event: 21.185 Executing VM operation: ICBufferFull -Event: 21.185 Executing VM operation: ICBufferFull done -Event: 21.240 Executing VM operation: ICBufferFull -Event: 21.240 Executing VM operation: ICBufferFull done -Event: 21.277 Executing VM operation: ICBufferFull -Event: 21.277 Executing VM operation: ICBufferFull done - -Memory protections (20 events): -Event: 9.544 Protecting memory [0x0000000304b30000,0x0000000304b3c000] with protection modes 3 -Event: 9.544 Protecting memory [0x0000000300c34000,0x0000000300c40000] with protection modes 3 -Event: 9.545 Protecting memory [0x0000000300a28000,0x0000000300a34000] with protection modes 3 -Event: 9.545 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 3 -Event: 9.557 Protecting memory [0x000000017e240000,0x000000017e24c000] with protection modes 3 -Event: 14.562 Protecting memory [0x0000000174c38000,0x0000000174c44000] with protection modes 3 -Event: 14.562 Protecting memory [0x00000001741fc000,0x0000000174208000] with protection modes 3 -Event: 14.562 Protecting memory [0x0000000174e44000,0x0000000174e50000] with protection modes 3 -Event: 16.522 Protecting memory [0x0000000173ff0000,0x0000000173ffc000] with protection modes 3 -Event: 21.113 Protecting memory [0x0000000173ff0000,0x0000000173ffc000] with protection modes 0 -Event: 21.113 Protecting memory [0x00000001741fc000,0x0000000174208000] with protection modes 0 -Event: 21.114 Protecting memory [0x0000000174c38000,0x0000000174c44000] with protection modes 0 -Event: 21.114 Protecting memory [0x0000000174e44000,0x0000000174e50000] with protection modes 0 -Event: 21.114 Protecting memory [0x000000017e240000,0x000000017e24c000] with protection modes 0 -Event: 21.169 Protecting memory [0x000000030081c000,0x0000000300828000] with protection modes 0 -Event: 21.222 Protecting memory [0x000000017f894000,0x000000017f8a0000] with protection modes 0 -Event: 21.254 Protecting memory [0x0000000300c28000,0x0000000300c34000] with protection modes 0 -Event: 21.260 Protecting memory [0x00000003040f4000,0x0000000304100000] with protection modes 0 -Event: 21.261 Protecting memory [0x000000030450c000,0x0000000304518000] with protection modes 0 -Event: 21.265 Protecting memory [0x0000000304718000,0x0000000304724000] with protection modes 0 - -Nmethod flushes (20 events): -Event: 7.362 Thread 0x0000000afcd82a00 flushing osr nmethod 0x000000011dccf790 -Event: 7.362 Thread 0x0000000afcd82a00 flushing osr nmethod 0x000000011dcd1190 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dcd2b90 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dd3e910 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011dd4cb90 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd7a10 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd8190 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd8a90 -Event: 7.362 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ddd9490 -Event: 7.991 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011ce31990 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d095f90 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d1be990 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2c1310 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2c2910 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d2cd910 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d31ab90 -Event: 7.992 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d32d490 -Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d4b0790 -Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d5c2f90 -Event: 7.993 Thread 0x0000000afcd82a00 flushing nmethod 0x000000011d60e790 - -Events (20 events): -Event: 9.544 Thread 0x0000000af68a5800 Thread exited: 0x0000000af68a5800 -Event: 9.544 Thread 0x0000000af6861800 Thread exited: 0x0000000af6861800 -Event: 9.545 Thread 0x0000000af6861200 Thread exited: 0x0000000af6861200 -Event: 9.545 Thread 0x0000000af6860c00 Thread exited: 0x0000000af6860c00 -Event: 9.557 Thread 0x0000000af6860000 Thread exited: 0x0000000af6860000 -Event: 14.562 Thread 0x0000000afb471200 Thread exited: 0x0000000afb471200 -Event: 14.563 Thread 0x0000000afb470c00 Thread exited: 0x0000000afb470c00 -Event: 14.563 Thread 0x0000000af816aa00 Thread exited: 0x0000000af816aa00 -Event: 16.522 Thread 0x0000000af8169800 Thread exited: 0x0000000af8169800 -Event: 21.113 Thread 0x0000000afcd82400 Thread added: 0x0000000af68a7600 -Event: 21.113 Thread 0x0000000afcd82400 Thread added: 0x0000000af8169800 -Event: 21.114 Thread 0x0000000afcd82400 Thread added: 0x0000000af816aa00 -Event: 21.114 Thread 0x0000000af816aa00 Thread added: 0x0000000afb470c00 -Event: 21.114 Thread 0x0000000af816aa00 Thread added: 0x0000000afb471200 -Event: 21.169 Thread 0x0000000afcd83600 Thread added: 0x0000000af6860c00 -Event: 21.222 Thread 0x0000000af816aa00 Thread added: 0x0000000af5631e00 -Event: 21.254 Thread 0x0000000af816aa00 Thread added: 0x0000000af601c600 -Event: 21.260 Thread 0x0000000afb471200 Thread added: 0x0000000af601cc00 -Event: 21.261 Thread 0x0000000afb471200 Thread added: 0x0000000af601d200 -Event: 21.265 Thread 0x0000000afcd82400 Thread added: 0x0000000af601d800 - - -Dynamic libraries: -0x00000001021bc000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjli.dylib -0x00000001ab90b000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x0000000190c98000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x000000019430c000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x000000018e092000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x000000019c2cf000 /usr/lib/libSystem.B.dylib -0x00000001923c5000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00000002460dd000 /System/Library/PrivateFrameworks/CollectionViewCore.framework/Versions/A/CollectionViewCore -0x0000000199660000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x000000019ed38000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x000000019f092000 /System/Library/PrivateFrameworks/UserActivity.framework/Versions/A/UserActivity -0x000000019499c000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x000000024e8aa000 /System/Library/PrivateFrameworks/FrontBoard.framework/Versions/A/FrontBoard -0x00000001a674b000 /System/Library/PrivateFrameworks/FrontBoardServices.framework/Versions/A/FrontBoardServices -0x000000028a849000 /usr/lib/libRosetta.dylib -0x000000027e0fe000 /System/Library/PrivateFrameworks/UIIntelligenceSupport.framework/Versions/A/UIIntelligenceSupport -0x000000023afe6000 /System/Library/Frameworks/SwiftUICore.framework/Versions/A/SwiftUICore -0x00000002846b2000 /System/Library/PrivateFrameworks/WritingTools.framework/Versions/A/WritingTools -0x0000000283503000 /System/Library/PrivateFrameworks/WindowManagement.framework/Versions/A/WindowManagement -0x000000018dcf1000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x000000019df6a000 /usr/lib/libspindump.dylib -0x0000000192580000 /System/Library/Frameworks/UniformTypeIdentifiers.framework/Versions/A/UniformTypeIdentifiers -0x000000019b566000 /usr/lib/libbsm.0.dylib -0x0000000196d6a000 /usr/lib/libapp_launch_measurement.dylib -0x000000019605f000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x0000000196d6e000 /System/Library/PrivateFrameworks/CoreAutoLayout.framework/Versions/A/CoreAutoLayout -0x0000000198bab000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x0000000199fb3000 /usr/lib/liblangid.dylib -0x0000000199666000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x0000000192fe1000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x000000024e5e3000 /System/Library/PrivateFrameworks/FocusEngine.framework/Versions/A/FocusEngine -0x00000001935a8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x000000024c1de000 /System/Library/PrivateFrameworks/DesignLibrary.framework/Versions/A/DesignLibrary -0x00000001a4a13000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x000000019dd5f000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x0000000198b83000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x000000019608f000 /usr/lib/libDiagnosticMessagesClient.dylib -0x000000019c218000 /usr/lib/libz.1.dylib -0x00000001a8d3b000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x000000019964b000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00000001904e7000 /usr/lib/libicucore.A.dylib -0x00000001a02cf000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x000000019f040000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x000000027ed11000 /System/Library/PrivateFrameworks/UpdateCycle.framework/Versions/A/UpdateCycle -0x00000001bcf36000 /System/Library/PrivateFrameworks/TextInput.framework/Versions/A/TextInput -0x0000000192f23000 /usr/lib/libMobileGestalt.dylib -0x0000000199347000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00000001965c4000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00000001900c2000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00000001a4392000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x0000000196a39000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x000000018f910000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x0000000196183000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x000000019e512000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x0000000192f21000 /usr/lib/libenergytrace.dylib -0x00000001b041a000 /System/Library/PrivateFrameworks/RenderBox.framework/Versions/A/RenderBox -0x0000000190b3f000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00000001a479c000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x0000000196cfb000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00000002309d0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x0000000196db8000 /usr/lib/libxml2.2.dylib -0x000000019b447000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x000000018c3a8000 /usr/lib/libobjc.A.dylib -0x000000018c70e000 /usr/lib/libc++.1.dylib -0x00000001a4719000 /System/Library/Frameworks/Accessibility.framework/Versions/A/Accessibility -0x0000000193d02000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x000000018c83f000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00000001999f3000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x000000018f6e3000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00000002322da000 /System/Library/Frameworks/CoreTransferable.framework/Versions/A/CoreTransferable -0x0000000232816000 /System/Library/Frameworks/DataDetection.framework/Versions/A/DataDetection -0x000000023282b000 /System/Library/Frameworks/DeveloperToolsSupport.framework/Versions/A/DeveloperToolsSupport -0x00000001996a4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x000000023bf5b000 /System/Library/Frameworks/Symbols.framework/Versions/A/Symbols -0x00000001c215b000 /System/Library/PrivateFrameworks/AttributeGraph.framework/Versions/A/AttributeGraph -0x00000001a682a000 /System/Library/PrivateFrameworks/BoardServices.framework/Versions/A/BoardServices -0x00000001e691a000 /System/Library/PrivateFrameworks/FeatureFlags.framework/Versions/A/FeatureFlags -0x0000000257823000 /System/Library/PrivateFrameworks/Gestures.framework/Versions/A/Gestures -0x000000019c2d4000 /System/Library/PrivateFrameworks/SoftLinking.framework/Versions/A/SoftLinking -0x00000001d09ff000 /usr/lib/swift/libswiftAccelerate.dylib -0x00000001c6e76000 /usr/lib/swift/libswiftCompression.dylib -0x000000019fc37000 /usr/lib/swift/libswiftCore.dylib -0x00000001b9207000 /usr/lib/swift/libswiftCoreFoundation.dylib -0x00000001b9260000 /usr/lib/swift/libswiftCoreImage.dylib -0x00000001a64c6000 /usr/lib/swift/libswiftDispatch.dylib -0x00000001b9261000 /usr/lib/swift/libswiftIOKit.dylib -0x00000001c6e69000 /usr/lib/swift/libswiftMetal.dylib -0x00000001d61ee000 /usr/lib/swift/libswiftOSLog.dylib -0x00000001a938c000 /usr/lib/swift/libswiftObjectiveC.dylib -0x000000028ba8a000 /usr/lib/swift/libswiftObservation.dylib -0x00000001cc182000 /usr/lib/swift/libswiftQuartzCore.dylib -0x000000028bbc9000 /usr/lib/swift/libswiftSynchronization.dylib -0x00000001d09f0000 /usr/lib/swift/libswiftUniformTypeIdentifiers.dylib -0x00000001b9219000 /usr/lib/swift/libswiftXPC.dylib -0x000000028bc03000 /usr/lib/swift/libswift_Builtin_float.dylib -0x000000028bc06000 /usr/lib/swift/libswift_Concurrency.dylib -0x00000001a9390000 /usr/lib/swift/libswiftos.dylib -0x00000001bccc6000 /usr/lib/swift/libswiftsimd.dylib -0x000000019c4fb000 /usr/lib/libcompression.dylib -0x000000019ec98000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x0000000193eae000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x000000019d9e8000 /usr/lib/libate.dylib -0x000000019c2c9000 /usr/lib/system/libcache.dylib -0x000000019c284000 /usr/lib/system/libcommonCrypto.dylib -0x000000019c2af000 /usr/lib/system/libcompiler_rt.dylib -0x000000019c2a4000 /usr/lib/system/libcopyfile.dylib -0x000000018c548000 /usr/lib/system/libcorecrypto.dylib -0x000000018c641000 /usr/lib/system/libdispatch.dylib -0x000000018c3fc000 /usr/lib/system/libdyld.dylib -0x000000019c2bf000 /usr/lib/system/libkeymgr.dylib -0x000000019c267000 /usr/lib/system/libmacho.dylib -0x000000019b53e000 /usr/lib/system/libquarantine.dylib -0x000000019c2bc000 /usr/lib/system/libremovefile.dylib -0x0000000192fa4000 /usr/lib/system/libsystem_asl.dylib -0x000000018c4cd000 /usr/lib/system/libsystem_blocks.dylib -0x000000018c68b000 /usr/lib/system/libsystem_c.dylib -0x000000019c2b3000 /usr/lib/system/libsystem_collections.dylib -0x0000000199fa0000 /usr/lib/system/libsystem_configuration.dylib -0x0000000198b4a000 /usr/lib/system/libsystem_containermanager.dylib -0x000000019bcce000 /usr/lib/system/libsystem_coreservices.dylib -0x00000001907b9000 /usr/lib/system/libsystem_darwin.dylib -0x000000028bf59000 /usr/lib/system/libsystem_darwindirectory.dylib -0x000000019c2c0000 /usr/lib/system/libsystem_dnssd.dylib -0x000000028bf5d000 /usr/lib/system/libsystem_eligibility.dylib -0x000000018c688000 /usr/lib/system/libsystem_featureflags.dylib -0x000000018c80f000 /usr/lib/system/libsystem_info.dylib -0x000000019c228000 /usr/lib/system/libsystem_m.dylib -0x000000018c5f5000 /usr/lib/system/libsystem_malloc.dylib -0x0000000192f06000 /usr/lib/system/libsystem_networkextension.dylib -0x0000000190c2a000 /usr/lib/system/libsystem_notify.dylib -0x0000000199fa5000 /usr/lib/system/libsystem_sandbox.dylib -0x000000028bf67000 /usr/lib/system/libsystem_sanitizers.dylib -0x000000019c2b8000 /usr/lib/system/libsystem_secinit.dylib -0x000000018c7bc000 /usr/lib/system/libsystem_kernel.dylib -0x000000018c806000 /usr/lib/system/libsystem_platform.dylib -0x000000018c7f9000 /usr/lib/system/libsystem_pthread.dylib -0x0000000194b80000 /usr/lib/system/libsystem_symptoms.dylib -0x000000018c527000 /usr/lib/system/libsystem_trace.dylib -0x000000028bf6f000 /usr/lib/system/libsystem_trial.dylib -0x000000019c292000 /usr/lib/system/libunwind.dylib -0x000000018c4d1000 /usr/lib/system/libxpc.dylib -0x000000018c7a1000 /usr/lib/libc++abi.dylib -0x000000028b79a000 /usr/lib/libswiftPrespecialized.dylib -0x000000019c2d1000 /usr/lib/libfakelink.dylib -0x0000000192b4a000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x000000019c325000 /usr/lib/libarchive.2.dylib -0x00000001a24a7000 /System/Library/Frameworks/Combine.framework/Versions/A/Combine -0x0000000246102000 /System/Library/PrivateFrameworks/CollectionsInternal.framework/Versions/A/CollectionsInternal -0x000000026d281000 /System/Library/PrivateFrameworks/ReflectionInternal.framework/Versions/A/ReflectionInternal -0x000000026e10d000 /System/Library/PrivateFrameworks/RuntimeInternal.framework/Versions/A/RuntimeInternal -0x00000001b60ae000 /usr/lib/swift/libswiftDarwin.dylib -0x000000028bbd1000 /usr/lib/swift/libswiftSystem.dylib -0x000000028bc90000 /usr/lib/swift/libswift_DarwinFoundation1.dylib -0x000000028bd71000 /usr/lib/swift/libswift_StringProcessing.dylib -0x0000000190abe000 /System/Library/PrivateFrameworks/CoreServicesInternal.framework/Versions/A/CoreServicesInternal -0x000000019c29c000 /usr/lib/liboah.dylib -0x00000001a87d9000 /System/Library/PrivateFrameworks/DiskImages.framework/Versions/A/DiskImages -0x00000001b5b70000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x0000000196d32000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00000001907c4000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00000001960f7000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x000000019bcd6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x000000019c41d000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x0000000194af9000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x000000018cd89000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x000000019d83e000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x0000000196d40000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x000000019c4b2000 /usr/lib/libapple_nghttp2.dylib -0x0000000194713000 /usr/lib/libsqlite3.dylib -0x00000001a8a21000 /System/Library/PrivateFrameworks/AppSupport.framework/Versions/A/AppSupport -0x00000001a0969000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x0000000194a8d000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00000001a931a000 /System/Library/PrivateFrameworks/StreamingZip.framework/Versions/A/StreamingZip -0x0000000194b89000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x000000019c26b000 /usr/lib/system/libkxld.dylib -0x0000000241963000 /System/Library/PrivateFrameworks/AppleKeyStore.framework/Versions/A/AppleKeyStore -0x000000028a52c000 /usr/lib/libCoreEntitlements.dylib -0x0000000263153000 /System/Library/PrivateFrameworks/MessageSecurity.framework/Versions/A/MessageSecurity -0x00000001946f7000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00000001a58c7000 /System/Library/PrivateFrameworks/SymptomDiagnosticReporter.framework/Versions/A/SymptomDiagnosticReporter -0x000000019da82000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x000000019bcb5000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x000000019b54e000 /usr/lib/libcoretls.dylib -0x000000019d8b5000 /usr/lib/libcoretls_cfhelpers.dylib -0x000000019c4f4000 /usr/lib/libpam.2.dylib -0x000000019d927000 /usr/lib/libxar.1.dylib -0x000000019d8b7000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x0000000278bb9000 /System/Library/PrivateFrameworks/SwiftASN1Internal.framework/Versions/A/SwiftASN1Internal -0x000000019d936000 /usr/lib/libutil.dylib -0x0000000199fae000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x000000019b236000 /System/Library/PrivateFrameworks/IOMobileFramebuffer.framework/Versions/A/IOMobileFramebuffer -0x00000001b90cd000 /System/Library/PrivateFrameworks/LoggingSupport.framework/Versions/A/LoggingSupport -0x00000001a09cc000 /System/Library/PrivateFrameworks/MobileAsset.framework/Versions/A/MobileAsset -0x00000001a58d7000 /System/Library/PrivateFrameworks/PowerLog.framework/Versions/A/PowerLog -0x000000023ae78000 /System/Library/Frameworks/SwiftData.framework/Versions/A/SwiftData -0x0000000196f65000 /System/Library/PrivateFrameworks/UserManagement.framework/Versions/A/UserManagement -0x0000000192a75000 /usr/lib/libboringssl.dylib -0x0000000194b6e000 /usr/lib/libdns_services.dylib -0x00000001b8104000 /usr/lib/libquic.dylib -0x000000019fbc7000 /usr/lib/libusrtcp.dylib -0x0000000242755000 /System/Library/PrivateFrameworks/AtomicsInternal.framework/Versions/A/AtomicsInternal -0x00000001dea7a000 /System/Library/PrivateFrameworks/InternalSwiftProtobuf.framework/Versions/A/InternalSwiftProtobuf -0x000000028ba61000 /usr/lib/swift/libswiftDistributed.dylib -0x0000000199fac000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x0000000242ee8000 /System/Library/PrivateFrameworks/BiomeLibrary.framework/Versions/A/BiomeLibrary -0x00000001c7e89000 /System/Library/PrivateFrameworks/BiomeStreams.framework/Versions/A/BiomeStreams -0x00000001c4054000 /System/Library/PrivateFrameworks/BiomeFoundation.framework/Versions/A/BiomeFoundation -0x00000001cdf4a000 /System/Library/PrivateFrameworks/BiomePubSub.framework/Versions/A/BiomePubSub -0x00000001aa346000 /System/Library/PrivateFrameworks/ProactiveSupport.framework/Versions/A/ProactiveSupport -0x000000019d896000 /usr/lib/liblzma.5.dylib -0x00000001a8d3c000 /System/Library/PrivateFrameworks/InternationalTextSearch.framework/Versions/A/InternationalTextSearch -0x00000001c0558000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreSupport.framework/Versions/A/SoftwareUpdateCoreSupport -0x00000001c890d000 /System/Library/PrivateFrameworks/SoftwareUpdateCoreConnect.framework/Versions/A/SoftwareUpdateCoreConnect -0x00000001a88d7000 /System/Library/PrivateFrameworks/RemoteServiceDiscovery.framework/Versions/A/RemoteServiceDiscovery -0x00000001c014f000 /System/Library/PrivateFrameworks/MSUDataAccessor.framework/Versions/A/MSUDataAccessor -0x00000001bb229000 /usr/lib/libbootpolicy.dylib -0x00000001a88ee000 /System/Library/PrivateFrameworks/RemoteXPC.framework/Versions/A/RemoteXPC -0x00000001c7d55000 /usr/lib/libFDR.dylib -0x00000001cdb6b000 /usr/lib/libamsupport.dylib -0x000000028a841000 /usr/lib/libReverseProxyDevice.dylib -0x000000024128c000 /System/Library/PrivateFrameworks/AppleDeviceQuerySupport.framework/Versions/A/AppleDeviceQuerySupport -0x00000001d0bff000 /usr/lib/libpartition2_dynamic.dylib -0x000000019c48e000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x000000028a3cd000 /usr/lib/libAppleArchive.dylib -0x000000019bcc1000 /usr/lib/libbz2.1.0.dylib -0x000000019618d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00000001a4776000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x000000019d96d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x000000018d289000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x0000000196e9f000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00000001a0570000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x000000019ec21000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00000001a0932000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00000001a092d000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00000001a0542000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x0000000199075000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00000001a85fd000 /System/Library/PrivateFrameworks/BackBoardServices.framework/Versions/A/BackBoardServices -0x0000000242e01000 /System/Library/PrivateFrameworks/BackBoardHIDEventFoundation.framework/Versions/A/BackBoardHIDEventFoundation -0x000000018f1f4000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x000000019e558000 /System/Library/Frameworks/VideoToolbox.framework/Versions/A/VideoToolbox -0x000000019c4f2000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x000000026c2ed000 /System/Library/PrivateFrameworks/ProDisplayLibrary.framework/Versions/A/ProDisplayLibrary -0x00000001ac3dd000 /System/Library/PrivateFrameworks/IOSurfaceAccelerator.framework/Versions/A/IOSurfaceAccelerator -0x0000000198ba0000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x0000000198eb8000 /System/Library/Frameworks/CoreMedia.framework/Versions/A/CoreMedia -0x0000000192fbc000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x000000019e510000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00000001b91e7000 /usr/lib/swift/libswiftCoreAudio.dylib -0x00000001d4a47000 /usr/lib/swift/libswiftCoreMedia.dylib -0x000000028bbeb000 /usr/lib/swift/libswiftVideoToolbox.dylib -0x00000001cdfa8000 /System/Library/PrivateFrameworks/BiomeStorage.framework/Versions/A/BiomeStorage -0x000000025d7b3000 /System/Library/PrivateFrameworks/IntelligencePlatformLibrary.framework/Versions/A/IntelligencePlatformLibrary -0x000000026b5ec000 /System/Library/PrivateFrameworks/PoirotSchematizer.framework/Versions/A/PoirotSchematizer -0x0000000243730000 /System/Library/PrivateFrameworks/BiomeSync.framework/Versions/A/BiomeSync -0x0000000242ece000 /System/Library/PrivateFrameworks/BiomeDSL.framework/Versions/A/BiomeDSL -0x000000026b656000 /System/Library/PrivateFrameworks/PoirotUDFs.framework/Versions/A/PoirotUDFs -0x000000026b5bb000 /System/Library/PrivateFrameworks/PoirotSQLite.framework/Versions/A/PoirotSQLite -0x00000001a6d47000 /System/Library/PrivateFrameworks/CoreTime.framework/Versions/A/CoreTime -0x000000019c30a000 /usr/lib/libiconv.2.dylib -0x000000019c266000 /usr/lib/libcharset.1.dylib -0x000000028bcd1000 /usr/lib/swift/libswift_RegexParser.dylib -0x0000000244d41000 /System/Library/PrivateFrameworks/CascadeSets.framework/Versions/A/CascadeSets -0x00000001a0b3c000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x000000019e2f8000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x000000019dcda000 /usr/lib/libexpat.1.dylib -0x000000019eaf4000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x000000019eb20000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x000000019ec0a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x000000019e33d000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x000000019ebb0000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x000000019eba7000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x0000000253f91000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libllvm-flatbuffers.dylib -0x000000024e89d000 /System/Library/PrivateFrameworks/FramePacing.framework/Versions/A/FramePacing -0x00000002309c2000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x000000024f9de000 /System/Library/PrivateFrameworks/GPUCompiler.framework/Versions/32023/Libraries/libGPUCompilerUtils.dylib -0x00000001a68f7000 /System/Library/PrivateFrameworks/GraphicsServices.framework/Versions/A/GraphicsServices -0x00000001aa4d2000 /System/Library/PrivateFrameworks/ASEProcessing.framework/Versions/A/ASEProcessing -0x00000001da249000 /System/Library/PrivateFrameworks/Symbolication.framework/Versions/A/Symbolication -0x000000026b1d8000 /System/Library/PrivateFrameworks/PhotosensitivityProcessing.framework/Versions/A/PhotosensitivityProcessing -0x000000026e346000 /System/Library/PrivateFrameworks/SILManager.framework/Versions/A/SILManager -0x000000019df0c000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x0000000230a21000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00000002309e4000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x0000000230bb0000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00000002309ed000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00000002309e1000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00000002309ca000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x000000024e7a8000 /System/Library/PrivateFrameworks/FontServices.framework/Versions/A/FontServices -0x000000019df1c000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00000001969e6000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x000000028ad51000 /usr/lib/libhvf.dylib -0x0000000268010000 /System/Library/PrivateFrameworks/ParsingInternal.framework/Versions/A/ParsingInternal -0x000000024e7ac000 /System/Library/PrivateFrameworks/FontServices.framework/libXTFontStaticRegistryData.dylib -0x0000000199eeb000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x000000019bc20000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x000000019b5f1000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x000000019ba29000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x000000019b83e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x000000019ba5b000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x0000000234b59000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSFunctions.framework/Versions/A/MPSFunctions -0x0000000234b3a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSBenchmarkLoop.framework/Versions/A/MPSBenchmarkLoop -0x0000000234b6e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/Frameworks/MPSHost.framework/Versions/A/MPSHost -0x000000018d0a5000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00000001be372000 /System/Library/PrivateFrameworks/IOAccelMemoryInfo.framework/Versions/A/IOAccelMemoryInfo -0x00000001cc578000 /System/Library/PrivateFrameworks/kperf.framework/Versions/A/kperf -0x00000001b91e3000 /System/Library/PrivateFrameworks/GPURawCounter.framework/Versions/A/GPURawCounter -0x00000001a6c30000 /System/Library/PrivateFrameworks/CoreSymbolication.framework/Versions/A/CoreSymbolication -0x00000001b915a000 /System/Library/PrivateFrameworks/MallocStackLogging.framework/Versions/A/MallocStackLogging -0x00000001a6c0d000 /System/Library/PrivateFrameworks/DebugSymbols.framework/Versions/A/DebugSymbols -0x00000001cadde000 /System/Library/PrivateFrameworks/OSAnalytics.framework/Versions/A/OSAnalytics -0x000000024c695000 /System/Library/PrivateFrameworks/DeviceRecovery.framework/Versions/A/DeviceRecovery -0x0000000232970000 /System/Library/Frameworks/ExtensionFoundation.framework/Versions/A/ExtensionFoundation -0x000000019df5d000 /System/Library/PrivateFrameworks/AppServerSupport.framework/Versions/A/AppServerSupport -0x0000000233cd8000 /System/Library/Frameworks/LightweightCodeRequirements.framework/Versions/A/LightweightCodeRequirements -0x000000019dbd9000 /System/Library/PrivateFrameworks/PlugInKit.framework/Versions/A/PlugInKit -0x000000019b45f000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x000000019dcff000 /System/Library/PrivateFrameworks/IconFoundation.framework/Versions/A/IconFoundation -0x000000025a6b5000 /System/Library/PrivateFrameworks/IconRendering.framework/Versions/A/IconRendering -0x000000026e2be000 /System/Library/PrivateFrameworks/SFSymbols.framework/Versions/A/SFSymbols -0x000000018f318000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00000001a04aa000 /System/Library/PrivateFrameworks/AudioSession.framework/Versions/A/AudioSession -0x00000001ab71e000 /System/Library/Frameworks/CoreMIDI.framework/Versions/A/CoreMIDI -0x00000001a0529000 /usr/lib/libAudioStatistics.dylib -0x0000000198e8f000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x000000019eace000 /usr/lib/libAudioToolboxUtility.dylib -0x00000001bf998000 /usr/lib/swift/libswiftCoreMIDI.dylib -0x00000001a093b000 /System/Library/PrivateFrameworks/perfdata.framework/Versions/A/perfdata -0x00000001b8201000 /System/Library/PrivateFrameworks/SystemPolicy.framework/Versions/A/SystemPolicy -0x00000001a07e0000 /usr/lib/libSMC.dylib -0x00000001a0949000 /usr/lib/libperfcheck.dylib -0x0000000242815000 /System/Library/PrivateFrameworks/AudioAnalytics.framework/Versions/A/AudioAnalytics -0x000000028bc94000 /usr/lib/swift/libswift_DarwinFoundation2.dylib -0x000000028bc95000 /usr/lib/swift/libswift_DarwinFoundation3.dylib -0x00000001de777000 /System/Library/Frameworks/OSLog.framework/Versions/A/OSLog -0x00000001b918c000 /usr/lib/libmis.dylib -0x000000019de1d000 /System/Library/PrivateFrameworks/MediaExperience.framework/Versions/A/MediaExperience -0x00000001a0278000 /System/Library/PrivateFrameworks/AudioSession.framework/libSessionUtility.dylib -0x000000019ec10000 /System/Library/PrivateFrameworks/CMCaptureCore.framework/Versions/A/CMCaptureCore -0x00000001a57c1000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x000000019b2cb000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00000001b1141000 /System/Library/PrivateFrameworks/HID.framework/Versions/A/HID -0x000000024ba92000 /System/Library/PrivateFrameworks/CoreUtilsExtras.framework/Versions/A/CoreUtilsExtras -0x000000025a60a000 /System/Library/PrivateFrameworks/IO80211.framework/Versions/A/IO80211 -0x00000001a21fa000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x0000000242403000 /System/Library/PrivateFrameworks/ArgumentParserInternal.framework/Versions/A/ArgumentParserInternal -0x000000019da79000 /usr/lib/libIOReport.dylib -0x000000019b46e000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x000000018da69000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x000000019b5d9000 /usr/lib/libgermantok.dylib -0x000000019a0d7000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00000001a59c9000 /System/Library/PrivateFrameworks/MediaKit.framework/Versions/A/MediaKit -0x00000001a5914000 /System/Library/Frameworks/DiscRecording.framework/Versions/A/DiscRecording -0x000000019dcf5000 /usr/lib/libheimdal-asn1.dylib -0x00000001a92f0000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x0000000196d05000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x0000000196d13000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x0000000230f34000 /System/Library/Frameworks/AVFAudio.framework/Versions/A/AVFAudio -0x00000001b2112000 /usr/lib/libAccessibility.dylib -0x000000025e35a000 /System/Library/PrivateFrameworks/IsolatedCoreAudioClient.framework/Versions/A/IsolatedCoreAudioClient -0x000000027c194000 /System/Library/PrivateFrameworks/Tightbeam.framework/Versions/A/Tightbeam -0x0000000247990000 /System/Library/PrivateFrameworks/CoreAudioOrchestration.framework/Versions/A/CoreAudioOrchestration -0x00000001d1152000 /System/Library/PrivateFrameworks/AFKUser.framework/Versions/A/AFKUser -0x00000001a4a63000 /System/Library/Frameworks/MediaAccessibility.framework/Versions/A/MediaAccessibility -0x00000002803b3000 /System/Library/PrivateFrameworks/VideoToolboxParavirtualizationSupport.framework/Versions/A/VideoToolboxParavirtualizationSupport -0x000000019dc91000 /System/Library/PrivateFrameworks/AppleVA.framework/Versions/A/AppleVA -0x000000019eba2000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x000000019eb82000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x000000019ebaa000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x000000024225a000 /System/Library/PrivateFrameworks/AppleMobileFileIntegrity.framework/Versions/A/AppleMobileFileIntegrity -0x000000028a8ce000 /usr/lib/libTLE.dylib -0x00000001efa7d000 /System/Library/PrivateFrameworks/ConfigProfileHelper.framework/Versions/A/ConfigProfileHelper -0x00000001e488a000 /usr/lib/libedit.3.dylib -0x000000025f8da000 /System/Library/PrivateFrameworks/MIL.framework/Versions/A/MIL -0x00000001be335000 /usr/lib/libncurses.5.4.dylib -0x000000019b200000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x000000018dd85000 /usr/lib/libCRFSuite.dylib -0x0000000199fb5000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x000000019b578000 /usr/lib/libmecab.dylib -0x000000019c485000 /usr/lib/libThaiTokenizer.dylib -0x00000001dec42000 /System/Library/PrivateFrameworks/HIDDisplay.framework/Versions/A/HIDDisplay -0x000000019ebdc000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00000001a08c1000 /usr/lib/libcups.2.dylib -0x00000001a0957000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00000001a05b8000 /usr/lib/libresolv.9.dylib -0x000000019df71000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00000001a92ef000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00000001a09bb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00000001b200d000 /System/Library/PrivateFrameworks/AXCoreUtilities.framework/Versions/A/AXCoreUtilities -0x000000028a2e7000 /usr/lib/libAXSafeCategoryBundle.dylib -0x000000019b0c6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x000000019c5d2000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x000000019b5dc000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x000000019c4cb000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x000000019c5cd000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x000000019a0de000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x000000018db7e000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00000001a0920000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x0000000196f24000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x000000019b542000 /usr/lib/libCheckFix.dylib -0x0000000196091000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x000000025b0d2000 /System/Library/PrivateFrameworks/InstalledContentLibrary.framework/Versions/A/InstalledContentLibrary -0x0000000190aff000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00000001cc67f000 /System/Library/PrivateFrameworks/MobileSystemServices.framework/Versions/A/MobileSystemServices -0x000000019d93a000 /usr/lib/libxslt.1.dylib -0x000000019b501000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00000001a8999000 /usr/lib/libcurl.4.dylib -0x000000028ab07000 /usr/lib/libcrypto.46.dylib -0x000000028b727000 /usr/lib/libssl.48.dylib -0x00000001a866f000 /System/Library/Frameworks/LDAP.framework/Versions/A/LDAP -0x00000001a86ab000 /System/Library/PrivateFrameworks/TrustEvaluationAgent.framework/Versions/A/TrustEvaluationAgent -0x00000001a05d5000 /usr/lib/libsasl2.2.dylib -0x00000001b60ad000 /usr/lib/swift/libswiftCoreGraphics.dylib -0x00000001a515f000 /usr/lib/swift/libswiftFoundation.dylib -0x00000001ef439000 /usr/lib/swift/libswiftSwiftOnoneSupport.dylib -0x000000028be06000 /usr/lib/swift/libswiftsys_time.dylib -0x00000001daa06000 /System/Library/PrivateFrameworks/CoreMaterial.framework/Versions/A/CoreMaterial -0x000000028bb4c000 /usr/lib/swift/libswiftSpatial.dylib -0x000000028a8cb000 /usr/lib/libSpatial.dylib -0x000000028a205000 /System/Library/SubFrameworks/UIUtilities.framework/Versions/A/UIUtilities -0x0000000107d58000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/server/libjvm.dylib -0x00000001021e4000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjimage.dylib -0x0000000102214000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libinstrument.dylib -0x0000000102264000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjava.dylib -0x00000001023ec000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libzip.dylib -0x0000000102414000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnio.dylib -0x000000010775c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libnet.dylib -0x0000000107780000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libverify.dylib -0x00000001023d8000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement.dylib -0x0000000107824000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libmanagement_ext.dylib -0x0000000107838000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libjaas.dylib -0x000000010784c000 /Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home/lib/libextnet.dylib -0x0000000147dd0000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/libcomet-766714755951389016.dylib -0x000000010789c000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/liblz4-java-15423878368099207039.dylib -0x0000000107948000 /Users/andy/git/apache/datafusion-comet/spark/target/tmp/snappy-1.1.10-afedc8a6-4493-4c3c-aed3-c280414ad816-libsnappyjava.dylib - - -VM Arguments: -jvm_args: -Djava.awt.headless=true -Djava.io.tmpdir=/Users/andy/git/apache/datafusion-comet/spark/target/tmp -Dlog4j.configurationFile=file:src/test/resources/log4j2.properties -Dbasedir=/Users/andy/git/apache/datafusion-comet/spark -javaagent:/Users/andy/.m2/repository/org/jacoco/org.jacoco.agent/0.8.11/org.jacoco.agent-0.8.11-runtime.jar=destfile=/Users/andy/git/apache/datafusion-comet/spark/target/jacoco.exec -ea -Xmx4g -Xss4m -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false -java_command: org.scalatest.tools.Runner -R /Users/andy/git/apache/datafusion-comet/spark/target/classes /Users/andy/git/apache/datafusion-comet/spark/target/test-classes -l org.apache.comet.IntegrationTestSuite -s org.apache.comet.exec.CometNativeShuffleSuite -oD -f /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/SparkTestSuite.txt -u /Users/andy/git/apache/datafusion-comet/spark/target/surefire-reports/. -java_class_path (initial): /Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/ -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 12 {product} {ergonomic} - uint ConcGCThreads = 5 {product} {ergonomic} - uint G1ConcRefinementThreads = 20 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - bool IgnoreUnrecognizedVMOptions = true {product} {command line} - size_t InitialHeapSize = 1610612736 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4294967296 {product} {command line} - size_t MaxNewSize = 2575302656 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 7602992 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122027624 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4294967296 {manageable} {ergonomic} - intx ThreadStackSize = 4096 {pd product} {command line} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - bool UseNUMA = false {product} {ergonomic} - bool UseNUMAInterleaving = false {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/temurin-17.jdk/Contents/Home -CLASSPATH=/Users/andy/git/apache/datafusion-comet/spark/target/test-classes:/Users/andy/git/apache/datafusion-comet/spark/target/classes:/Users/andy/git/apache/datafusion-comet/common/target/classes:/Users/andy/.m2/repository/org/scala-lang/modules/scala-collection-compat_2.12/2.12.0/scala-collection-compat_2.12-2.12.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/rocksdb/rocksdbjni/8.3.2/rocksdbjni-8.3.2.jar:/Users/andy/.m2/repository/com/univocity/univocity-parsers/2.9.1/univocity-parsers-2.9.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sketch_2.12/3.5.8/spark-sketch_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-tags_2.12/3.5.8/spark-tags_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/orc/orc-core/1.9.8/orc-core-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/orc/orc-shims/1.9.8/orc-shims-1.9.8.jar:/Users/andy/.m2/repository/org/jetbrains/annotations/17.0.0/annotations-17.0.0.jar:/Users/andy/.m2/repository/org/threeten/threeten-extra/1.7.1/threeten-extra-1.7.1.jar:/Users/andy/.m2/repository/org/apache/orc/orc-mapreduce/1.9.8/orc-mapreduce-1.9.8-shaded-protobuf.jar:/Users/andy/.m2/repository/org/apache/hive/hive-storage-api/2.8.1/hive-storage-api-2.8.1.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-databind/2.15.2/jackson-databind-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-core/2.15.2/jackson-core-2.15.2.jar:/Users/andy/.m2/repository/org/apache/xbean/xbean-asm9-shaded/4.23/xbean-asm9-shaded-4.23.jar:/Users/andy/.m2/repository/org/scala-lang/scala-library/2.12.18/scala-library-2.12.18.jar:/Users/andy/.m2/repository/org/scala-lang/scala-reflect/2.12.18/scala-reflect-2.12.18.jar:/Users/andy/.m2/repository/com/google/protobuf/protobuf-java/3.25.5/protobuf-java-3.25.5.jar:/Users/andy/.m2/repository/org/scalatest/scalatest_2.12/3.2.16/scalatest_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-core_2.12/3.2.16/scalatest-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-compatible/3.2.16/scalatest-compatible-3.2.16.jar:/Users/andy/.m2/repository/org/scalactic/scalactic_2.12/3.2.16/scalactic_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-featurespec_2.12/3.2.16/scalatest-featurespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-flatspec_2.12/3.2.16/scalatest-flatspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-freespec_2.12/3.2.16/scalatest-freespec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funsuite_2.12/3.2.16/scalatest-funsuite_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-funspec_2.12/3.2.16/scalatest-funspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-propspec_2.12/3.2.16/scalatest-propspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-refspec_2.12/3.2.16/scalatest-refspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-wordspec_2.12/3.2.16/scalatest-wordspec_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-diagrams_2.12/3.2.16/scalatest-diagrams_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-matchers-core_2.12/3.2.16/scalatest-matchers-core_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-shouldmatchers_2.12/3.2.16/scalatest-shouldmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatest/scalatest-mustmatchers_2.12/3.2.16/scalatest-mustmatchers_2.12-3.2.16.jar:/Users/andy/.m2/repository/org/scalatestplus/junit-4-13_2.12/3.2.16.0/junit-4-13_2.12-3.2.16.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-core_2.12/3.5.8/spark-core_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/avro/avro/1.11.5/avro-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-mapred/1.11.5/avro-mapred-1.11.5.jar:/Users/andy/.m2/repository/org/apache/avro/avro-ipc/1.11.5/avro-ipc-1.11.5.jar:/Users/andy/.m2/repository/org/tukaani/xz/1.9/xz-1.9.jar:/Users/andy/.m2/repository/com/twitter/chill_2.12/0.10.0/chill_2.12-0.10.0.jar:/Users/andy/.m2/repository/com/esotericsoftware/kryo-shaded/4.0.2/kryo-shaded-4.0.2.jar:/Users/andy/.m2/repository/com/esotericsoftware/minlog/1.3.0/minlog-1.3.0.jar:/Users/andy/.m2/repository/org/objenesis/objenesis/2.5.1/objenesis-2.5.1.jar:/Users/andy/.m2/repository/com/twitter/chill-java/0.10.0/chill-java-0.10.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-api/3.3.4/hadoop-client-api-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-runtime/3.3.4/hadoop-client-runtime-3.3.4.jar:/Users/andy/.m2/repository/org/apache/spark/spark-launcher_2.12/3.5.8/spark-launcher_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-kvstore_2.12/3.5.8/spark-kvstore_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-common_2.12/3.5.8/spark-network-common_2.12-3.5.8.jar:/Users/andy/.m2/repository/com/google/crypto/tink/tink/1.9.0/tink-1.9.0.jar:/Users/andy/.m2/repository/com/google/code/gson/gson/2.10.1/gson-2.10.1.jar:/Users/andy/.m2/repository/org/apache/spark/spark-network-shuffle_2.12/3.5.8/spark-network-shuffle_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-unsafe_2.12/3.5.8/spark-unsafe_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/apache/spark/spark-common-utils_2.12/3.5.8/spark-common-utils_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/slf4j/jul-to-slf4j/2.0.7/jul-to-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/slf4j/jcl-over-slf4j/2.0.7/jcl-over-slf4j-2.0.7.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-slf4j2-impl/2.20.0/log4j-slf4j2-impl-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-api/2.20.0/log4j-api-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-core/2.20.0/log4j-core-2.20.0.jar:/Users/andy/.m2/repository/org/apache/logging/log4j/log4j-1.2-api/2.20.0/log4j-1.2-api-2.20.0.jar:/Users/andy/.m2/repository/javax/activation/activation/1.1.1/activation-1.1.1.jar:/Users/andy/.m2/repository/org/apache/curator/curator-recipes/2.13.0/curator-recipes-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-framework/2.13.0/curator-framework-2.13.0.jar:/Users/andy/.m2/repository/org/apache/curator/curator-client/2.13.0/curator-client-2.13.0.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper/3.6.3/zookeeper-3.6.3.jar:/Users/andy/.m2/repository/org/apache/zookeeper/zookeeper-jute/3.6.3/zookeeper-jute-3.6.3.jar:/Users/andy/.m2/repository/jakarta/servlet/jakarta.servlet-api/4.0.3/jakarta.servlet-api-4.0.3.jar:/Users/andy/.m2/repository/commons-codec/commons-codec/1.16.1/commons-codec-1.16.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-compress/1.23.0/commons-compress-1.23.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-lang3/3.12.0/commons-lang3-3.12.0.jar:/Users/andy/.m2/repository/org/apache/commons/commons-math3/3.6.1/commons-math3-3.6.1.jar:/Users/andy/.m2/repository/org/apache/commons/commons-text/1.10.0/commons-text-1.10.0.jar:/Users/andy/.m2/repository/commons-io/commons-io/2.16.1/commons-io-2.16.1.jar:/Users/andy/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/andy/.m2/repository/org/apache/commons/commons-collections4/4.4/commons-collections4-4.4.jar:/Users/andy/.m2/repository/com/ning/compress-lzf/1.1.2/compress-lzf-1.1.2.jar:/Users/andy/.m2/repository/org/xerial/snappy/snappy-java/1.1.10.5/snappy-java-1.1.10.5.jar:/Users/andy/.m2/repository/org/lz4/lz4-java/1.8.0/lz4-java-1.8.0.jar:/Users/andy/.m2/repository/com/github/luben/zstd-jni/1.5.5-4/zstd-jni-1.5.5-4.jar:/Users/andy/.m2/repository/org/roaringbitmap/RoaringBitmap/0.9.45/RoaringBitmap-0.9.45.jar:/Users/andy/.m2/repository/org/roaringbitmap/shims/0.9.45/shims-0.9.45.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-xml_2.12/2.1.0/scala-xml_2.12-2.1.0.jar:/Users/andy/.m2/repository/org/json4s/json4s-jackson_2.12/3.7.0-M11/json4s-jackson_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-core_2.12/3.7.0-M11/json4s-core_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-ast_2.12/3.7.0-M11/json4s-ast_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/json4s/json4s-scalap_2.12/3.7.0-M11/json4s-scalap_2.12-3.7.0-M11.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-client/2.47/jersey-client-2.47.jar:/Users/andy/.m2/repository/jakarta/ws/rs/jakarta.ws.rs-api/2.1.6/jakarta.ws.rs-api-2.1.6.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/jakarta.inject/2.6.1/jakarta.inject-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-common/2.47/jersey-common-2.47.jar:/Users/andy/.m2/repository/jakarta/annotation/jakarta.annotation-api/1.3.5/jakarta.annotation-api-1.3.5.jar:/Users/andy/.m2/repository/org/glassfish/hk2/osgi-resource-locator/1.0.3/osgi-resource-locator-1.0.3.jar:/Users/andy/.m2/repository/org/glassfish/jersey/core/jersey-server/2.47/jersey-server-2.47.jar:/Users/andy/.m2/repository/jakarta/validation/jakarta.validation-api/2.0.2/jakarta.validation-api-2.0.2.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet/2.47/jersey-container-servlet-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/containers/jersey-container-servlet-core/2.47/jersey-container-servlet-core-2.47.jar:/Users/andy/.m2/repository/org/glassfish/jersey/inject/jersey-hk2/2.47/jersey-hk2-2.47.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-locator/2.6.1/hk2-locator-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/external/aopalliance-repackaged/2.6.1/aopalliance-repackaged-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-api/2.6.1/hk2-api-2.6.1.jar:/Users/andy/.m2/repository/org/glassfish/hk2/hk2-utils/2.6.1/hk2-utils-2.6.1.jar:/Users/andy/.m2/repository/org/javassist/javassist/3.30.2-GA/javassist-3.30.2-GA.jar:/Users/andy/.m2/repository/io/netty/netty-all/4.1.96.Final/netty-all-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-buffer/4.1.96.Final/netty-buffer-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec/4.1.96.Final/netty-codec-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http/4.1.96.Final/netty-codec-http-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-http2/4.1.96.Final/netty-codec-http2-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-codec-socks/4.1.96.Final/netty-codec-socks-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-common/4.1.96.Final/netty-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler/4.1.96.Final/netty-handler-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-unix-common/4.1.96.Final/netty-transport-native-unix-common-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-handler-proxy/4.1.96.Final/netty-handler-proxy-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-resolver/4.1.96.Final/netty-resolver-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport/4.1.96.Final/netty-transport-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-epoll/4.1.96.Final/netty-transport-classes-epoll-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-classes-kqueue/4.1.96.Final/netty-transport-classes-kqueue-4.1.96.Final.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-x86_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-epoll/4.1.96.Final/netty-transport-native-epoll-4.1.96.Final-linux-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-aarch_64.jar:/Users/andy/.m2/repository/io/netty/netty-transport-native-kqueue/4.1.96.Final/netty-transport-native-kqueue-4.1.96.Final-osx-x86_64.jar:/Users/andy/.m2/repository/com/clearspring/analytics/stream/2.9.6/stream-2.9.6.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-core/4.2.19/metrics-core-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jvm/4.2.19/metrics-jvm-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-json/4.2.19/metrics-json-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-graphite/4.2.19/metrics-graphite-4.2.19.jar:/Users/andy/.m2/repository/io/dropwizard/metrics/metrics-jmx/4.2.19/metrics-jmx-4.2.19.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/module/jackson-module-scala_2.12/2.15.2/jackson-module-scala_2.12-2.15.2.jar:/Users/andy/.m2/repository/com/thoughtworks/paranamer/paranamer/2.8/paranamer-2.8.jar:/Users/andy/.m2/repository/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar:/Users/andy/.m2/repository/oro/oro/2.0.8/oro-2.0.8.jar:/Users/andy/.m2/repository/net/razorvine/pickle/1.3/pickle-1.3.jar:/Users/andy/.m2/repository/net/sf/py4j/py4j/0.10.9.7/py4j-0.10.9.7.jar:/Users/andy/.m2/repository/org/apache/commons/commons-crypto/1.1.0/commons-crypto-1.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-catalyst_2.12/3.5.8/spark-catalyst_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql-api_2.12/3.5.8/spark-sql-api_2.12-3.5.8.jar:/Users/andy/.m2/repository/org/scala-lang/modules/scala-parser-combinators_2.12/2.3.0/scala-parser-combinators_2.12-2.3.0.jar:/Users/andy/.m2/repository/org/antlr/antlr4-runtime/4.9.3/antlr4-runtime-4.9.3.jar:/Users/andy/.m2/repository/org/codehaus/janino/janino/3.1.9/janino-3.1.9.jar:/Users/andy/.m2/repository/org/codehaus/janino/commons-compiler/3.1.9/commons-compiler-3.1.9.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-java/3.3.0/datasketches-java-3.3.0.jar:/Users/andy/.m2/repository/org/apache/datasketches/datasketches-memory/2.1.0/datasketches-memory-2.1.0.jar:/Users/andy/.m2/repository/org/apache/spark/spark-sql_2.12/3.5.8/spark-sql_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/spark/spark-hadoop-cloud_2.12/3.5.8/spark-hadoop-cloud_2.12-3.5.8-tests.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aws/3.3.4/hadoop-aws-3.3.4.jar:/Users/andy/.m2/repository/com/amazonaws/aws-java-sdk-bundle/1.12.262/aws-java-sdk-bundle-1.12.262.jar:/Users/andy/.m2/repository/org/wildfly/openssl/wildfly-openssl/1.0.7.Final/wildfly-openssl-1.0.7.Final.jar:/Users/andy/.m2/repository/com/google/cloud/bigdataoss/gcs-connector/hadoop3-2.2.14/gcs-connector-hadoop3-2.2.14-shaded.jar:/Users/andy/.m2/repository/joda-time/joda-time/2.12.5/joda-time-2.12.5.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/core/jackson-annotations/2.15.2/jackson-annotations-2.15.2.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/dataformat/jackson-dataformat-cbor/2.15.2/jackson-dataformat-cbor-2.15.2.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar:/Users/andy/.m2/repository/org/apache/httpcomponents/httpcore/4.4.16/httpcore-4.4.16.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure/3.3.4/hadoop-azure-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-storage/7.0.1/azure-storage-7.0.1.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-keyvault-core/1.0.0/azure-keyvault-core-1.0.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-cloud-storage/3.3.4/hadoop-cloud-storage-3.3.4.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-aliyun/3.3.4/hadoop-aliyun-3.3.4.jar:/Users/andy/.m2/repository/com/aliyun/oss/aliyun-sdk-oss/3.13.0/aliyun-sdk-oss-3.13.0.jar:/Users/andy/.m2/repository/org/jdom/jdom2/2.0.6/jdom2-2.0.6.jar:/Users/andy/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/andy/.m2/repository/stax/stax-api/1.0.1/stax-api-1.0.1.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-core/4.5.10/aliyun-java-sdk-core-4.5.10.jar:/Users/andy/.m2/repository/org/ini4j/ini4j/0.5.4/ini4j-0.5.4.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-api/0.33.0/opentracing-api-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-util/0.33.0/opentracing-util-0.33.0.jar:/Users/andy/.m2/repository/io/opentracing/opentracing-noop/0.33.0/opentracing-noop-0.33.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-ram/3.1.0/aliyun-java-sdk-ram-3.1.0.jar:/Users/andy/.m2/repository/com/aliyun/aliyun-java-sdk-kms/2.11.0/aliyun-java-sdk-kms-2.11.0.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-azure-datalake/3.3.4/hadoop-azure-datalake-3.3.4.jar:/Users/andy/.m2/repository/com/microsoft/azure/azure-data-lake-store-sdk/2.3.9/azure-data-lake-store-sdk-2.3.9.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-openstack/3.3.4/hadoop-openstack-3.3.4.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util/9.4.58.v20250814/jetty-util-9.4.58.v20250814.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-util-ajax/9.4.58.v20250814/jetty-util-ajax-9.4.58.v20250814.jar:/Users/andy/.m2/repository/junit/junit/4.13.2/junit-4.13.2.jar:/Users/andy/.m2/repository/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar:/Users/andy/.m2/repository/com/google/guava/guava/33.2.1-jre/guava-33.2.1-jre.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/andy/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-column/1.13.1/parquet-column-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-encoding/1.13.1/parquet-encoding-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-format-structures/1.13.1/parquet-format-structures-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-common/1.13.1/parquet-common-1.13.1.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-jackson/1.13.1/parquet-jackson-1.13.1.jar:/Users/andy/.m2/repository/io/airlift/aircompressor/0.21/aircompressor-0.21.jar:/Users/andy/.m2/repository/commons-pool/commons-pool/1.6/commons-pool-1.6.jar:/Users/andy/.m2/repository/org/slf4j/slf4j-api/2.0.7/slf4j-api-2.0.7.jar:/Users/andy/.m2/repository/org/apache/yetus/audience-annotations/0.13.0/audience-annotations-0.13.0.jar:/Users/andy/.m2/repository/org/apache/parquet/parquet-hadoop/1.13.1/parquet-hadoop-1.13.1-tests.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-unsafe/18.3.0/arrow-memory-unsafe-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-memory-core/18.3.0/arrow-memory-core-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-c-data/18.3.0/arrow-c-data-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-vector/18.3.0/arrow-vector-18.3.0.jar:/Users/andy/.m2/repository/org/apache/arrow/arrow-format/18.3.0/arrow-format-18.3.0.jar:/Users/andy/.m2/repository/com/fasterxml/jackson/datatype/jackson-datatype-jsr310/2.18.3/jackson-datatype-jsr310-2.18.3.jar:/Users/andy/.m2/repository/com/google/flatbuffers/flatbuffers-java/25.2.10/flatbuffers-java-25.2.10.jar:/Users/andy/.m2/repository/org/apache/hadoop/hadoop-client-minicluster/3.3.4/hadoop-client-minicluster-3.3.4.jar:/Users/andy/.m2/repository/org/testcontainers/minio/1.21.0/minio-1.21.0.jar:/Users/andy/.m2/repository/org/testcontainers/testcontainers/1.21.0/testcontainers-1.21.0.jar:/Users/andy/.m2/repository/org/rnorth/duct-tape/duct-tape/1.0.8/duct-tape-1.0.8.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-api/3.4.2/docker-java-api-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport-zerodep/3.4.2/docker-java-transport-zerodep-3.4.2.jar:/Users/andy/.m2/repository/com/github/docker-java/docker-java-transport/3.4.2/docker-java-transport-3.4.2.jar:/Users/andy/.m2/repository/net/java/dev/jna/jna/5.13.0/jna-5.13.0.jar:/Users/andy/.m2/repository/software/amazon/awssdk/s3/2.31.51/s3-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-xml-protocol/2.31.51/aws-xml-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/protocol-core/2.31.51/protocol-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/arns/2.31.51/arns-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/profiles/2.31.51/profiles-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/crt-core/2.31.51/crt-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth/2.31.51/http-auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/identity-spi/2.31.51/identity-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-spi/2.31.51/http-auth-spi-2.31.51.jar:/Users/andy/.m2/repository/org/reactivestreams/reactive-streams/1.0.4/reactive-streams-1.0.4.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws/2.31.51/http-auth-aws-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums/2.31.51/checksums-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/checksums-spi/2.31.51/checksums-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries-spi/2.31.51/retries-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sdk-core/2.31.51/sdk-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/retries/2.31.51/retries-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/auth/2.31.51/auth-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-auth-aws-eventstream/2.31.51/http-auth-aws-eventstream-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar:/Users/andy/.m2/repository/software/amazon/awssdk/http-client-spi/2.31.51/http-client-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/regions/2.31.51/regions-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/annotations/2.31.51/annotations-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/utils/2.31.51/utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-core/2.31.51/aws-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/metrics-spi/2.31.51/metrics-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/json-utils/2.31.51/json-utils-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/third-party-jackson-core/2.31.51/third-party-jackson-core-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/endpoints-spi/2.31.51/endpoints-spi-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/apache-client/2.31.51/apache-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/netty-nio-client/2.31.51/netty-nio-client-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/sts/2.31.51/sts-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-query-protocol/2.31.51/aws-query-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/dynamodb/2.31.51/dynamodb-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/aws-json-protocol/2.31.51/aws-json-protocol-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/glue/2.31.51/glue-2.31.51.jar:/Users/andy/.m2/repository/software/amazon/awssdk/kms/2.31.51/kms-2.31.51.jar:/Users/andy/.m2/repository/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-server/9.4.53.v20231009/jetty-server-9.4.53.v20231009.jar:/Users/andy/.m2/repository/javax/servlet/javax.servlet-api/3.1.0/javax.servlet-api-3.1.0.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-http/9.4.53.v20231009/jetty-http-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-io/9.4.53.v20231009/jetty-io-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-servlet/9.4.53.v20231009/jetty-servlet-9.4.53.v20231009.jar:/Users/andy/.m2/repository/org/eclipse/jetty/jetty-security/9.4.53.v20231009/jetty-security-9.4.53.v20231009.jar -PATH=/Users/andy/.local/bin:/opt/homebrew/bin:/opt/homebrew/sbin:/usr/local/bin:/System/Cryptexes/App/usr/bin:/usr/bin:/bin:/usr/sbin:/sbin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/local/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/bin:/var/run/com.apple.security.cryptexd/codex.system/bootstrap/usr/appleinternal/bin:/opt/pmk/env/global/bin:/Library/Apple/usr/bin:/Users/andy/.cargo/bin:/Users/andy/Library/Application Support/JetBrains/Toolbox/scripts:/Users/andy/.claude/plugins/cache/claude-plugins-official/github/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/commit-commands/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/rust-analyzer-lsp/1.0.0/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/frontend-design/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/superpowers/5.0.7/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-review/unknown/bin:/Users/andy/.claude/plugins/cache/claude-plugins-official/code-simplifier/1.0.0/bin -SHELL=/bin/zsh -LANG=en_US.UTF-8 -TERM=xterm-256color -TMPDIR=/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/ - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.dylib, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.dylib, mask=11100110000111110111111111111111, flags=SA_RESTART|SA_SIGINFO, unblocked - - -Periodic native trim disabled - - ---------------- S Y S T E M --------------- - -OS: -uname: Darwin 25.3.0 Darwin Kernel Version 25.3.0: Wed Jan 28 20:54:55 PST 2026; root:xnu-12377.91.3~2/RELEASE_ARM64_T6031 arm64 -OS uptime: 4 days 1:42 hours -rlimit (soft/hard): STACK 8176k/65520k , CORE 0k/infinity , NPROC 10666/16000 , NOFILE 1048576/infinity , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK infinity/infinity , RSS infinity/infinity -load average: 8.23 7.63 5.76 - -CPU: total 28 (initial active 28) 0x61:0x0:0x72015832:0, fp, simd, crc, lse -machdep.cpu.brand_string:Apple M3 Ultra -hw.cachelinesize:128 -hw.l1icachesize:131072 -hw.l1dcachesize:65536 -hw.l2cachesize:4194304 - -Memory: 16k page, physical 100663296k(3080768k free), swap 2097152k(1070592k free) - -vm_info: OpenJDK 64-Bit Server VM (17.0.17+10) for bsd-aarch64 JRE (17.0.17+10), built on Oct 21 2025 00:00:00 by "admin" with clang Apple LLVM 15.0.0 (clang-1500.1.0.2.5) - -END. From e6c3a42ae69239fe424117f0096e7cc009fe831f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:30:41 -0600 Subject: [PATCH 14/21] refactor: deduplicate executePlan logic and address code review findings - Extract shared execution logic into execute_plan_impl with OutputMode enum - Replace stringly-typed handle mode detection with HANDLE_SCAN_SOURCE constant - Remove no-op catch/throw in CometExecIterator.nextHandle() - Remove unnecessary #![allow(dead_code)] from batch_stash module - Remove unnecessary @volatile from stashMode field --- native/core/src/execution/batch_stash.rs | 7 +- native/core/src/execution/jni_api.rs | 538 +++++++----------- native/core/src/execution/planner.rs | 8 +- .../org/apache/comet/CometExecIterator.scala | 14 +- 4 files changed, 209 insertions(+), 358 deletions(-) diff --git a/native/core/src/execution/batch_stash.rs b/native/core/src/execution/batch_stash.rs index d34f54f885..1c6be690dd 100644 --- a/native/core/src/execution/batch_stash.rs +++ b/native/core/src/execution/batch_stash.rs @@ -18,11 +18,12 @@ //! Global registry for passing RecordBatch values between native execution contexts //! via opaque u64 handles, without Arrow FFI serialization. -// These items will be called from JNI and other modules added in subsequent tasks. -#![allow(dead_code)] - use arrow::record_batch::RecordBatch; use once_cell::sync::Lazy; + +/// Scan source name indicating the input uses the batch stash handle path. +/// Must match the value set in CometNativeShuffleWriter.getNativePlan(). +pub(crate) const HANDLE_SCAN_SOURCE: &str = "ShuffleWriterInputHandle"; use std::collections::HashMap; use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Mutex; diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index ae70c14e0d..f24e4c7f33 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -618,12 +618,200 @@ fn prepare_output( } /// Stash the output RecordBatch in the BatchStash and return the handle. -/// Used when output feeds directly into another native plan. fn stash_output(output_batch: RecordBatch) -> CometResult { let handle = crate::execution::batch_stash::stash(output_batch); Ok(handle as jlong) } +/// How to handle output batches from the execution plan. +enum OutputMode<'a> { + /// Export via Arrow FFI to the provided addresses. + Ffi { + array_addrs: JLongArray<'a>, + schema_addrs: JLongArray<'a>, + validate: bool, + }, + /// Stash in BatchStash and return handle. + Stash, +} + +impl OutputMode<'_> { + fn handle_batch(&self, env: &mut Env, batch: RecordBatch) -> CometResult { + match self { + OutputMode::Ffi { + array_addrs, + schema_addrs, + validate, + } => { + // Safety: JLongArray is a raw JNI reference that remains valid for the + // duration of the JNI call. We reborrow it here since prepare_output + // only reads from it. + let array_addrs = unsafe { JLongArray::from_raw(env, array_addrs.as_raw()) }; + let schema_addrs = unsafe { JLongArray::from_raw(env, schema_addrs.as_raw()) }; + prepare_output(env, array_addrs, schema_addrs, batch, *validate) + } + OutputMode::Stash => stash_output(batch), + } + } +} + +/// Shared execution logic for `executePlan` and `executePlanBatchHandle`. +fn execute_plan_impl( + env: &mut Env, + stage_id: jint, + partition: jint, + exec_context: &mut ExecutionContext, + output_mode: &OutputMode, +) -> CometResult { + let tracing_enabled = exec_context.tracing_enabled; + let owned_label; + let tracing_label = if tracing_enabled { + owned_label = exec_context.tracing_event_name.clone(); + owned_label.as_str() + } else { + "" + }; + + let result = with_trace(tracing_label, tracing_enabled, || { + let exec_context_id = exec_context.id; + + // Initialize the execution stream. + // Because we don't know if input arrays are dictionary-encoded when we create + // query plan, we need to defer stream initialization to first time execution. + if exec_context.root_op.is_none() { + let start = Instant::now(); + let planner = PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) + .with_exec_id(exec_context_id); + let (scans, shuffle_scans, root_op) = planner.create_plan( + &exec_context.spark_plan, + &mut exec_context.input_sources.clone(), + exec_context.partition_count, + )?; + let physical_plan_time = start.elapsed(); + + exec_context.plan_creation_time += physical_plan_time; + exec_context.scans = scans; + exec_context.shuffle_scans = shuffle_scans; + + if exec_context.explain_native { + let formatted_plan_str = + DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); + info!("Comet native query plan:\n{formatted_plan_str:}"); + } + + let task_ctx = exec_context.session_ctx.task_ctx(); + // Each Comet native execution corresponds to a single Spark partition, + // so we should always execute partition 0. + let stream = root_op.native_plan.execute(0, task_ctx)?; + + if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { + // No JVM data sources -- spawn onto tokio so the executor + // thread parks in blocking_recv instead of busy-polling. + let (tx, rx) = mpsc::channel(2); + let mut stream = stream; + get_runtime().spawn(async move { + let result = std::panic::AssertUnwindSafe(async { + while let Some(batch) = stream.next().await { + if tx.send(batch).await.is_err() { + break; + } + } + }) + .catch_unwind() + .await; + + if let Err(panic) = result { + let msg = match panic.downcast_ref::<&str>() { + Some(s) => s.to_string(), + None => match panic.downcast_ref::() { + Some(s) => s.clone(), + None => "unknown panic".to_string(), + }, + }; + let _ = tx + .send(Err(DataFusionError::Execution(format!( + "native panic: {msg}" + )))) + .await; + } + }); + exec_context.batch_receiver = Some(rx); + } else { + exec_context.stream = Some(stream); + } + exec_context.root_op = Some(root_op); + } else { + pull_input_batches(exec_context)?; + } + + if let Some(rx) = &mut exec_context.batch_receiver { + match rx.blocking_recv() { + Some(Ok(batch)) => { + update_metrics(env, exec_context)?; + return output_mode.handle_batch(env, batch); + } + Some(Err(e)) => { + return Err(e.into()); + } + None => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + } + } + + // ScanExec path: busy-poll to interleave JVM batch pulls with stream polling + get_runtime().block_on(async { + loop { + let next_item = exec_context.stream.as_mut().unwrap().next(); + let poll_output = poll!(next_item); + + exec_context.poll_count_since_metrics_check += 1; + if exec_context.poll_count_since_metrics_check >= 100 { + exec_context.poll_count_since_metrics_check = 0; + if let Some(interval) = exec_context.metrics_update_interval { + let now = Instant::now(); + if now - exec_context.metrics_last_update_time >= interval { + update_metrics(env, exec_context)?; + exec_context.metrics_last_update_time = now; + } + } + if exec_context.tracing_enabled { + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + } + + match poll_output { + Poll::Ready(Some(output)) => { + return output_mode.handle_batch(env, output?); + } + Poll::Ready(None) => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + Poll::Pending => { + tokio::task::block_in_place(|| pull_input_batches(exec_context))?; + } + } + } + }) + }); + + if exec_context.tracing_enabled { + #[cfg(feature = "jemalloc")] + log_jemalloc_usage(); + log_memory_usage( + &exec_context.tracing_memory_metric_name, + total_reserved_for_thread(exec_context.rust_thread_id) as u64, + ); + } + + result +} + /// Pull the next input from JVM. Note that we cannot pull input batches in /// `ScanStream.poll_next` when the execution stream is polled for output. /// Because the input source could be another native execution stream, which @@ -657,188 +845,18 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( schema_addrs: JLongArray, ) -> jlong { try_unwrap_or_throw(&e, |env| { - // Retrieve the query let exec_context = get_execution_context(exec_context); - - let tracing_enabled = exec_context.tracing_enabled; - // Clone the label only when tracing is enabled. The clone is needed - // because the closure below mutably borrows exec_context. - let owned_label; - let tracing_label = if tracing_enabled { - owned_label = exec_context.tracing_event_name.clone(); - owned_label.as_str() - } else { - "" + let output_mode = OutputMode::Ffi { + array_addrs, + schema_addrs, + validate: exec_context.debug_native, }; - - let result = with_trace(tracing_label, tracing_enabled, || { - let exec_context_id = exec_context.id; - - // Initialize the execution stream. - // Because we don't know if input arrays are dictionary-encoded when we create - // query plan, we need to defer stream initialization to first time execution. - if exec_context.root_op.is_none() { - let start = Instant::now(); - let planner = - PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) - .with_exec_id(exec_context_id); - let (scans, shuffle_scans, root_op) = planner.create_plan( - &exec_context.spark_plan, - &mut exec_context.input_sources.clone(), - exec_context.partition_count, - )?; - let physical_plan_time = start.elapsed(); - - exec_context.plan_creation_time += physical_plan_time; - exec_context.scans = scans; - exec_context.shuffle_scans = shuffle_scans; - - if exec_context.explain_native { - let formatted_plan_str = - DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); - info!("Comet native query plan:\n{formatted_plan_str:}"); - } - - let task_ctx = exec_context.session_ctx.task_ctx(); - // Each Comet native execution corresponds to a single Spark partition, - // so we should always execute partition 0. - let stream = root_op.native_plan.execute(0, task_ctx)?; - - if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { - // No JVM data sources — spawn onto tokio so the executor - // thread parks in blocking_recv instead of busy-polling. - // - // Channel capacity of 2 allows the producer to work one batch - // ahead while the consumer processes the current one via JNI, - // without buffering excessive memory. Increasing this would - // trade memory for latency hiding if JNI/FFI overhead dominates; - // decreasing to 1 would serialize production and consumption. - let (tx, rx) = mpsc::channel(2); - let mut stream = stream; - get_runtime().spawn(async move { - let result = std::panic::AssertUnwindSafe(async { - while let Some(batch) = stream.next().await { - if tx.send(batch).await.is_err() { - break; - } - } - }) - .catch_unwind() - .await; - - if let Err(panic) = result { - let msg = match panic.downcast_ref::<&str>() { - Some(s) => s.to_string(), - None => match panic.downcast_ref::() { - Some(s) => s.clone(), - None => "unknown panic".to_string(), - }, - }; - let _ = tx - .send(Err(DataFusionError::Execution(format!( - "native panic: {msg}" - )))) - .await; - } - }); - exec_context.batch_receiver = Some(rx); - } else { - exec_context.stream = Some(stream); - } - exec_context.root_op = Some(root_op); - } else { - // Pull input batches - pull_input_batches(exec_context)?; - } - - if let Some(rx) = &mut exec_context.batch_receiver { - match rx.blocking_recv() { - Some(Ok(batch)) => { - update_metrics(env, exec_context)?; - return prepare_output( - env, - array_addrs, - schema_addrs, - batch, - exec_context.debug_native, - ); - } - Some(Err(e)) => { - return Err(e.into()); - } - None => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - } - } - - // ScanExec path: busy-poll to interleave JVM batch pulls with stream polling - get_runtime().block_on(async { - loop { - let next_item = exec_context.stream.as_mut().unwrap().next(); - let poll_output = poll!(next_item); - - // Only check time/tracing every 100 polls to reduce overhead - exec_context.poll_count_since_metrics_check += 1; - if exec_context.poll_count_since_metrics_check >= 100 { - exec_context.poll_count_since_metrics_check = 0; - if let Some(interval) = exec_context.metrics_update_interval { - let now = Instant::now(); - if now - exec_context.metrics_last_update_time >= interval { - update_metrics(env, exec_context)?; - exec_context.metrics_last_update_time = now; - } - } - if exec_context.tracing_enabled { - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - } - - match poll_output { - Poll::Ready(Some(output)) => { - return prepare_output( - env, - array_addrs, - schema_addrs, - output?, - exec_context.debug_native, - ); - } - Poll::Ready(None) => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - Poll::Pending => { - // JNI call to pull batches from JVM into ScanExec operators. - // block_in_place lets tokio move other tasks off this worker - // while we wait for JVM data. - tokio::task::block_in_place(|| pull_input_batches(exec_context))?; - } - } - } - }) - }); - - if exec_context.tracing_enabled { - #[cfg(feature = "jemalloc")] - log_jemalloc_usage(); - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - - result + execute_plan_impl(env, stage_id, partition, exec_context, &output_mode) }) } -/// Execute one step of the native query plan, stashing the output RecordBatch in the -/// BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) -/// or -1 for EOF. Used when the output feeds directly into another native plan. +/// Like executePlan but stashes the output RecordBatch and returns a handle instead of +/// exporting via Arrow FFI. Used when output feeds directly into another native plan. /// # Safety /// This function is inherently unsafe since it deals with raw pointers passed from JNI. #[no_mangle] @@ -850,170 +868,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlanBatchHandl exec_context: jlong, ) -> jlong { try_unwrap_or_throw(&e, |env| { - // Retrieve the query let exec_context = get_execution_context(exec_context); - - let tracing_enabled = exec_context.tracing_enabled; - // Clone the label only when tracing is enabled. The clone is needed - // because the closure below mutably borrows exec_context. - let owned_label; - let tracing_label = if tracing_enabled { - owned_label = exec_context.tracing_event_name.clone(); - owned_label.as_str() - } else { - "" - }; - - let result = with_trace(tracing_label, tracing_enabled, || { - let exec_context_id = exec_context.id; - - // Initialize the execution stream. - // Because we don't know if input arrays are dictionary-encoded when we create - // query plan, we need to defer stream initialization to first time execution. - if exec_context.root_op.is_none() { - let start = Instant::now(); - let planner = - PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) - .with_exec_id(exec_context_id); - let (scans, shuffle_scans, root_op) = planner.create_plan( - &exec_context.spark_plan, - &mut exec_context.input_sources.clone(), - exec_context.partition_count, - )?; - let physical_plan_time = start.elapsed(); - - exec_context.plan_creation_time += physical_plan_time; - exec_context.scans = scans; - exec_context.shuffle_scans = shuffle_scans; - - if exec_context.explain_native { - let formatted_plan_str = - DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); - info!("Comet native query plan:\n{formatted_plan_str:}"); - } - - let task_ctx = exec_context.session_ctx.task_ctx(); - // Each Comet native execution corresponds to a single Spark partition, - // so we should always execute partition 0. - let stream = root_op.native_plan.execute(0, task_ctx)?; - - if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { - // No JVM data sources — spawn onto tokio so the executor - // thread parks in blocking_recv instead of busy-polling. - // - // Channel capacity of 2 allows the producer to work one batch - // ahead while the consumer processes the current one via JNI, - // without buffering excessive memory. Increasing this would - // trade memory for latency hiding if JNI/FFI overhead dominates; - // decreasing to 1 would serialize production and consumption. - let (tx, rx) = mpsc::channel(2); - let mut stream = stream; - get_runtime().spawn(async move { - let result = std::panic::AssertUnwindSafe(async { - while let Some(batch) = stream.next().await { - if tx.send(batch).await.is_err() { - break; - } - } - }) - .catch_unwind() - .await; - - if let Err(panic) = result { - let msg = match panic.downcast_ref::<&str>() { - Some(s) => s.to_string(), - None => match panic.downcast_ref::() { - Some(s) => s.clone(), - None => "unknown panic".to_string(), - }, - }; - let _ = tx - .send(Err(DataFusionError::Execution(format!( - "native panic: {msg}" - )))) - .await; - } - }); - exec_context.batch_receiver = Some(rx); - } else { - exec_context.stream = Some(stream); - } - exec_context.root_op = Some(root_op); - } else { - // Pull input batches - pull_input_batches(exec_context)?; - } - - if let Some(rx) = &mut exec_context.batch_receiver { - match rx.blocking_recv() { - Some(Ok(batch)) => { - update_metrics(env, exec_context)?; - return stash_output(batch); - } - Some(Err(e)) => { - return Err(e.into()); - } - None => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - } - } - - // ScanExec path: busy-poll to interleave JVM batch pulls with stream polling - get_runtime().block_on(async { - loop { - let next_item = exec_context.stream.as_mut().unwrap().next(); - let poll_output = poll!(next_item); - - // Only check time/tracing every 100 polls to reduce overhead - exec_context.poll_count_since_metrics_check += 1; - if exec_context.poll_count_since_metrics_check >= 100 { - exec_context.poll_count_since_metrics_check = 0; - if let Some(interval) = exec_context.metrics_update_interval { - let now = Instant::now(); - if now - exec_context.metrics_last_update_time >= interval { - update_metrics(env, exec_context)?; - exec_context.metrics_last_update_time = now; - } - } - if exec_context.tracing_enabled { - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - } - - match poll_output { - Poll::Ready(Some(output)) => { - return stash_output(output?); - } - Poll::Ready(None) => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - Poll::Pending => { - // JNI call to pull batches from JVM into ScanExec operators. - // block_in_place lets tokio move other tasks off this worker - // while we wait for JVM data. - tokio::task::block_in_place(|| pull_input_batches(exec_context))?; - } - } - } - }) - }); - - if exec_context.tracing_enabled { - #[cfg(feature = "jemalloc")] - log_jemalloc_usage(); - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - - result + execute_plan_impl(env, stage_id, partition, exec_context, &OutputMode::Stash) }) } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 4e0b6f58b9..75683cb48f 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1304,11 +1304,9 @@ impl PhysicalPlanner { scan.arrow_ffi_safe, )?; - // Check if the scan source indicates handle mode (batch stash path). - // When the JVM sets source to "ShuffleWriterInputHandle", the input - // is a CometHandleBatchIterator and batches should be retrieved from - // the BatchStash instead of via Arrow FFI. - if scan.input_source_description == "ShuffleWriterInputHandle" { + if scan.input_source_description + == crate::execution::batch_stash::HANDLE_SCAN_SOURCE + { scan.handle_mode = true; } diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index d939504a33..1916436600 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -82,7 +82,7 @@ class CometExecIterator( private val cometTaskMemoryManager = new CometTaskMemoryManager(id, taskAttemptId) // When true, executePlan stashes output batches natively and returns handles // instead of exporting via Arrow FFI. Used when output feeds a native ShuffleWriter. - @volatile private var stashMode: Boolean = false + private var stashMode: Boolean = false private var pendingHandle: Long = -1L // Build a mixed array of iterators: CometShuffleBlockIterator for shuffle // scan indices, CometBatchIterator for regular scan indices. @@ -276,15 +276,11 @@ class CometExecIterator( } val ctx = TaskContext.get() - try { - val handle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) - if (handle == -1L) { - close() - } - handle - } catch { - case e: Throwable => throw e + val handle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) + if (handle == -1L) { + close() } + handle } def close(): Unit = synchronized { From 6cc0e78bee6c3dc62cc9955886561c7158b5880a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:38:58 -0600 Subject: [PATCH 15/21] refactor: use protobuf field for batch stash handle mode instead of scan source name --- native/core/src/execution/batch_stash.rs | 4 ---- native/core/src/execution/planner.rs | 6 +++--- native/proto/src/proto/operator.proto | 3 +++ .../comet/execution/shuffle/CometNativeShuffleWriter.scala | 6 ++++-- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/batch_stash.rs b/native/core/src/execution/batch_stash.rs index 1c6be690dd..b5ca53e2b1 100644 --- a/native/core/src/execution/batch_stash.rs +++ b/native/core/src/execution/batch_stash.rs @@ -20,10 +20,6 @@ use arrow::record_batch::RecordBatch; use once_cell::sync::Lazy; - -/// Scan source name indicating the input uses the batch stash handle path. -/// Must match the value set in CometNativeShuffleWriter.getNativePlan(). -pub(crate) const HANDLE_SCAN_SOURCE: &str = "ShuffleWriterInputHandle"; use std::collections::HashMap; use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Mutex; diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 75683cb48f..f8ec7cd9f4 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1295,6 +1295,8 @@ impl PhysicalPlanner { Some(inputs.remove(0)) }; + let use_batch_stash = scan.batch_stash_handle; + // The `ScanExec` operator will take actual arrays from Spark during execution let mut scan = ScanExec::new( self.exec_context_id, @@ -1304,9 +1306,7 @@ impl PhysicalPlanner { scan.arrow_ffi_safe, )?; - if scan.input_source_description - == crate::execution::batch_stash::HANDLE_SCAN_SOURCE - { + if use_batch_stash { scan.handle_mode = true; } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index fb438b26a4..df5d5b308a 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -84,6 +84,9 @@ message Scan { string source = 2; // Whether native code can assume ownership of batches that it receives bool arrow_ffi_safe = 3; + // When true, the input is a CometHandleBatchIterator and batches should be + // retrieved from the BatchStash instead of via Arrow FFI import. + bool batch_stash_handle = 4; } message ShuffleScan { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 9dad73fef9..96b48c4203 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -186,8 +186,10 @@ class CometNativeShuffleWriter[K, V]( dataFile: String, indexFile: String, useHandleMode: Boolean = false): Operator = { - val scanSource = if (useHandleMode) "ShuffleWriterInputHandle" else "ShuffleWriterInput" - val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource(scanSource) + val scanBuilder = OperatorOuterClass.Scan + .newBuilder() + .setSource("ShuffleWriterInput") + .setBatchStashHandle(useHandleMode) val opBuilder = OperatorOuterClass.Operator.newBuilder() val scanTypes = outputAttributes.flatten { attr => From c42be90ad5da9e48995b12ad08bfef657f12b384 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:41:53 -0600 Subject: [PATCH 16/21] chore: remove design docs from PR --- ...-04-12-batch-stash-shuffle-optimization.md | 1206 ----------------- ...batch-stash-shuffle-optimization-design.md | 254 ---- 2 files changed, 1460 deletions(-) delete mode 100644 docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md delete mode 100644 docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md diff --git a/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md b/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md deleted file mode 100644 index 404fcefc1c..0000000000 --- a/docs/superpowers/plans/2026-04-12-batch-stash-shuffle-optimization.md +++ /dev/null @@ -1,1206 +0,0 @@ -# Batch Stash Shuffle Optimization Implementation Plan - -> **For agentic workers:** REQUIRED SUB-SKILL: Use superpowers:subagent-driven-development (recommended) or superpowers:executing-plans to implement this plan task-by-task. Steps use checkbox (`- [ ]`) syntax for tracking. - -**Goal:** Avoid unnecessary Arrow FFI import/export when passing batches between a native child plan and a native ShuffleWriter by using an opaque batch handle passed through the JVM. - -**Architecture:** A native-side `BatchStash` registry stores `RecordBatch` values keyed by `u64` handles. The child plan stashes its output batch and returns the handle to the JVM. The JVM passes the handle to the shuffle writer's `ScanExec`, which retrieves the batch directly from the stash. This eliminates 4 FFI boundary crossings per batch, replacing them with 2 lightweight JNI calls passing a single `long`. - -**Tech Stack:** Rust (native), Scala/Java (JVM), JNI, Arrow RecordBatch - -**Spec:** `docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md` - ---- - -### Task 1: BatchStash (Rust) - -**Files:** -- Create: `native/core/src/execution/batch_stash.rs` -- Modify: `native/core/src/execution/mod.rs:19-33` - -- [ ] **Step 1: Write the BatchStash test** - -Create `native/core/src/execution/batch_stash.rs` with a test module: - -```rust -// 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. - -//! A global registry for passing RecordBatch values between native execution -//! contexts through the JVM without Arrow FFI serialization. - -use std::collections::HashMap; -use std::sync::atomic::{AtomicU64, Ordering}; -use std::sync::Mutex; - -use arrow::record_batch::RecordBatch; -use once_cell::sync::Lazy; - -static NEXT_HANDLE: AtomicU64 = AtomicU64::new(1); -static STASH: Lazy>> = Lazy::new(|| Mutex::new(HashMap::new())); - -/// Stash a RecordBatch and return a unique handle for later retrieval. -pub fn stash(batch: RecordBatch) -> u64 { - todo!() -} - -/// Remove and return the RecordBatch for the given handle. -/// Returns `None` if the handle is not found (already consumed or invalid). -pub fn take(handle: u64) -> Option { - todo!() -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::Int32Array; - use arrow::datatypes::{DataType, Field, Schema}; - use std::sync::Arc; - - fn make_batch(values: &[i32]) -> RecordBatch { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - let array = Int32Array::from(values.to_vec()); - RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap() - } - - #[test] - fn test_stash_and_take() { - let batch = make_batch(&[1, 2, 3]); - let handle = stash(batch); - assert!(handle > 0); - - let retrieved = take(handle).expect("batch should be in stash"); - assert_eq!(retrieved.num_rows(), 3); - assert_eq!( - retrieved - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .values(), - &[1, 2, 3] - ); - } - - #[test] - fn test_take_removes_entry() { - let batch = make_batch(&[10, 20]); - let handle = stash(batch); - - assert!(take(handle).is_some()); - assert!(take(handle).is_none(), "second take should return None"); - } - - #[test] - fn test_take_unknown_handle() { - assert!(take(999_999_999).is_none()); - } - - #[test] - fn test_handles_are_unique() { - let h1 = stash(make_batch(&[1])); - let h2 = stash(make_batch(&[2])); - assert_ne!(h1, h2); - - // Clean up - take(h1); - take(h2); - } -} -``` - -- [ ] **Step 2: Run test to verify it fails** - -Run: `cd native && cargo test --lib execution::batch_stash` -Expected: FAIL with `not yet implemented` - -- [ ] **Step 3: Implement stash and take** - -Replace the `todo!()` bodies in `native/core/src/execution/batch_stash.rs`: - -```rust -pub fn stash(batch: RecordBatch) -> u64 { - let handle = NEXT_HANDLE.fetch_add(1, Ordering::Relaxed); - STASH.lock().unwrap().insert(handle, batch); - handle -} - -pub fn take(handle: u64) -> Option { - STASH.lock().unwrap().remove(&handle) -} -``` - -- [ ] **Step 4: Add the module to mod.rs** - -In `native/core/src/execution/mod.rs`, add after line 18 (before existing pub mod declarations): - -```rust -pub(crate) mod batch_stash; -``` - -- [ ] **Step 5: Run tests to verify they pass** - -Run: `cd native && cargo test --lib execution::batch_stash` -Expected: all 4 tests PASS - -- [ ] **Step 6: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 7: Commit** - -```bash -git add native/core/src/execution/batch_stash.rs native/core/src/execution/mod.rs -git commit -m "feat: add BatchStash registry for native batch handle passing" -``` - ---- - -### Task 2: CometHandleBatchIterator (Java + Rust JNI bridge) - -**Files:** -- Create: `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` -- Create: `native/jni-bridge/src/handle_batch_iterator.rs` -- Modify: `native/jni-bridge/src/lib.rs:181-289` - -- [ ] **Step 1: Create CometHandleBatchIterator Java class** - -Create `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java`: - -```java -/* - * 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.comet; - -/** - * Iterator that passes opaque native batch handles between two native execution contexts through - * the JVM. Used when a native child plan feeds directly into a native ShuffleWriter, avoiding - * Arrow FFI export/import overhead. - * - *

Called from native ScanExec via JNI. The source CometExecIterator must be in stash mode. - */ -public class CometHandleBatchIterator { - private final CometExecIterator source; - - public CometHandleBatchIterator(CometExecIterator source) { - this.source = source; - } - - /** - * Get the next batch handle from the source iterator. - * - * @return a native batch handle (positive long), or -1 if no more batches. - */ - public long nextHandle() { - return source.nextHandle(); - } -} -``` - -- [ ] **Step 2: Create the Rust JNI bridge struct** - -Create `native/jni-bridge/src/handle_batch_iterator.rs`: - -```rust -// 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. - -use jni::signature::Primitive; -use jni::{ - errors::Result as JniResult, - objects::{JClass, JMethodID}, - signature::ReturnType, - strings::JNIString, - Env, -}; - -/// A struct that holds JNI methods for the JVM `CometHandleBatchIterator` class. -#[allow(dead_code)] // we need to keep references to Java items to prevent GC -pub struct CometHandleBatchIterator<'a> { - pub class: JClass<'a>, - pub method_next_handle: JMethodID, - pub method_next_handle_ret: ReturnType, -} - -impl<'a> CometHandleBatchIterator<'a> { - pub const JVM_CLASS: &'static str = "org/apache/comet/CometHandleBatchIterator"; - - pub fn new(env: &mut Env<'a>) -> JniResult> { - let class = env.find_class(JNIString::new(Self::JVM_CLASS))?; - - Ok(CometHandleBatchIterator { - class, - method_next_handle: env.get_method_id( - JNIString::new(Self::JVM_CLASS), - jni::jni_str!("nextHandle"), - jni::jni_sig!("()J"), - )?, - method_next_handle_ret: ReturnType::Primitive(Primitive::Long), - }) - } -} -``` - -- [ ] **Step 3: Register the new class in JVMClasses** - -In `native/jni-bridge/src/lib.rs`, add the module declaration after line 184 (`mod shuffle_block_iterator;`): - -```rust -mod handle_batch_iterator; -``` - -Add the import after line 189 (`use shuffle_block_iterator::CometShuffleBlockIterator;`): - -```rust -use handle_batch_iterator::CometHandleBatchIterator; -``` - -Add the field to the `JVMClasses` struct after line 216 (`pub comet_shuffle_block_iterator: CometShuffleBlockIterator<'a>,`): - -```rust - /// The CometHandleBatchIterator class. Used for passing batch handles between native plans. - pub comet_handle_batch_iterator: CometHandleBatchIterator<'a>, -``` - -Add initialization in the `JVMClasses::init` method after line 288 (`comet_shuffle_block_iterator: CometShuffleBlockIterator::new(env).unwrap(),`): - -```rust - comet_handle_batch_iterator: CometHandleBatchIterator::new(env).unwrap(), -``` - -- [ ] **Step 4: Verify it compiles** - -Run: `cd native && cargo build` -Expected: compilation succeeds (the Java class must exist on the classpath; if building native-only fails, run `make` from the project root first) - -- [ ] **Step 5: Commit** - -```bash -git add spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java \ - native/jni-bridge/src/handle_batch_iterator.rs \ - native/jni-bridge/src/lib.rs -git commit -m "feat: add CometHandleBatchIterator Java class and JNI bridge" -``` - ---- - -### Task 3: executePlanBatchHandle JNI function - -**Files:** -- Modify: `spark/src/main/scala/org/apache/comet/Native.scala:91-96` -- Modify: `native/core/src/execution/jni_api.rs:550-830` - -- [ ] **Step 1: Add the native method declaration in Native.scala** - -In `spark/src/main/scala/org/apache/comet/Native.scala`, add after the `executePlan` method (after line 96): - -```scala - /** - * Execute one step of the native query plan, stashing the output RecordBatch in the native - * BatchStash instead of exporting via Arrow FFI. Returns the stash handle (positive long) - * or -1 for EOF. Used when the output feeds directly into another native plan (e.g., - * native ShuffleWriter) to avoid unnecessary FFI round-trips. - * - * @param stage - * the stage ID, for informational purposes - * @param partition - * the partition ID, for informational purposes - * @param plan - * the address to native query plan. - * @return - * a batch stash handle (positive), or -1 for EOF. - */ - @native def executePlanBatchHandle( - stage: Int, - partition: Int, - plan: Long): Long -``` - -- [ ] **Step 2: Add the stash_output helper function in jni_api.rs** - -In `native/core/src/execution/jni_api.rs`, add after the `prepare_output` function (after line 618): - -```rust -/// Stash the output RecordBatch in the BatchStash and return the handle. -/// Used when output feeds directly into another native plan. -fn stash_output(output_batch: RecordBatch) -> CometResult { - let handle = crate::execution::batch_stash::stash(output_batch); - Ok(handle as jlong) -} -``` - -- [ ] **Step 3: Add the JNI function implementation** - -In `native/core/src/execution/jni_api.rs`, add after the `Java_org_apache_comet_Native_executePlan` function (after line 830): - -```rust -/// Like executePlan but stashes the output RecordBatch in the BatchStash and returns -/// the handle instead of exporting via Arrow FFI. Used for native-to-native batch passing. -/// # Safety -/// This function is inherently unsafe since it deals with raw pointers passed from JNI. -#[no_mangle] -pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlanBatchHandle( - e: EnvUnowned, - _class: JClass, - stage_id: jint, - partition: jint, - exec_context: jlong, -) -> jlong { - try_unwrap_or_throw(&e, |env| { - let exec_context = get_execution_context(exec_context); - - let tracing_enabled = exec_context.tracing_enabled; - let owned_label; - let tracing_label = if tracing_enabled { - owned_label = exec_context.tracing_event_name.clone(); - owned_label.as_str() - } else { - "" - }; - - let result = with_trace(tracing_label, tracing_enabled, || { - let exec_context_id = exec_context.id; - - // Initialize the execution stream on first call (same as executePlan) - if exec_context.root_op.is_none() { - let start = Instant::now(); - let planner = - PhysicalPlanner::new(Arc::clone(&exec_context.session_ctx), partition) - .with_exec_id(exec_context_id); - let (scans, shuffle_scans, root_op) = planner.create_plan( - &exec_context.spark_plan, - &mut exec_context.input_sources.clone(), - exec_context.partition_count, - )?; - let physical_plan_time = start.elapsed(); - - exec_context.plan_creation_time += physical_plan_time; - exec_context.scans = scans; - exec_context.shuffle_scans = shuffle_scans; - - if exec_context.explain_native { - let formatted_plan_str = - DisplayableExecutionPlan::new(root_op.native_plan.as_ref()).indent(true); - info!("Comet native query plan:\n{formatted_plan_str:}"); - } - - let task_ctx = exec_context.session_ctx.task_ctx(); - let stream = root_op.native_plan.execute(0, task_ctx)?; - - if exec_context.scans.is_empty() && exec_context.shuffle_scans.is_empty() { - let (tx, rx) = mpsc::channel(2); - let mut stream = stream; - get_runtime().spawn(async move { - let result = std::panic::AssertUnwindSafe(async { - while let Some(batch) = stream.next().await { - if tx.send(batch).await.is_err() { - break; - } - } - }) - .catch_unwind() - .await; - - if let Err(panic) = result { - let msg = match panic.downcast_ref::<&str>() { - Some(s) => s.to_string(), - None => match panic.downcast_ref::() { - Some(s) => s.clone(), - None => "unknown panic".to_string(), - }, - }; - let _ = tx - .send(Err(DataFusionError::Execution(format!( - "native panic: {msg}" - )))) - .await; - } - }); - exec_context.batch_receiver = Some(rx); - } else { - exec_context.stream = Some(stream); - } - exec_context.root_op = Some(root_op); - } else { - pull_input_batches(exec_context)?; - } - - if let Some(rx) = &mut exec_context.batch_receiver { - match rx.blocking_recv() { - Some(Ok(batch)) => { - update_metrics(env, exec_context)?; - return stash_output(batch); - } - Some(Err(e)) => { - return Err(e.into()); - } - None => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - } - } - - // ScanExec path: busy-poll - get_runtime().block_on(async { - loop { - let next_item = exec_context.stream.as_mut().unwrap().next(); - let poll_output = poll!(next_item); - - exec_context.poll_count_since_metrics_check += 1; - if exec_context.poll_count_since_metrics_check >= 100 { - exec_context.poll_count_since_metrics_check = 0; - if let Some(interval) = exec_context.metrics_update_interval { - let now = Instant::now(); - if now - exec_context.metrics_last_update_time >= interval { - update_metrics(env, exec_context)?; - exec_context.metrics_last_update_time = now; - } - } - if exec_context.tracing_enabled { - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - } - - match poll_output { - Poll::Ready(Some(output)) => { - return stash_output(output?); - } - Poll::Ready(None) => { - log_plan_metrics(exec_context, stage_id, partition); - return Ok(-1); - } - Poll::Pending => { - tokio::task::block_in_place(|| pull_input_batches(exec_context))?; - } - } - } - }) - }); - - if exec_context.tracing_enabled { - #[cfg(feature = "jemalloc")] - log_jemalloc_usage(); - log_memory_usage( - &exec_context.tracing_memory_metric_name, - total_reserved_for_thread(exec_context.rust_thread_id) as u64, - ); - } - - result - }) -} -``` - -- [ ] **Step 4: Verify it compiles** - -Run: `cd native && cargo build` -Expected: compilation succeeds - -- [ ] **Step 5: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 6: Commit** - -```bash -git add spark/src/main/scala/org/apache/comet/Native.scala \ - native/core/src/execution/jni_api.rs -git commit -m "feat: add executePlanBatchHandle JNI function for stash-mode output" -``` - ---- - -### Task 4: CometExecIterator stash mode - -**Files:** -- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` - -- [ ] **Step 1: Add stash mode fields** - -In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add after line 81 (`private val cometTaskMemoryManager = new CometTaskMemoryManager(id, taskAttemptId)`): - -```scala - // When true, executePlan stashes output batches natively and returns handles - // instead of exporting via Arrow FFI. Used when output feeds a native ShuffleWriter. - @volatile private var stashMode: Boolean = false - private var pendingHandle: Long = -1L -``` - -- [ ] **Step 2: Add enableStashMode and nextHandle methods** - -In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add after the `close()` method (after line 254, before the `traceMemoryUsage` method): - -```scala - /** Enable stash mode. Must be called before iteration begins. */ - def enableStashMode(): Unit = { - stashMode = true - } - - /** - * In stash mode, advance the native plan and return the batch handle. - * Returns a positive handle, or -1 for EOF. - */ - def nextHandle(): Long = { - if (closed) return -1L - - if (pendingHandle >= 0) { - val h = pendingHandle - pendingHandle = -1L - return h - } - - val ctx = TaskContext.get() - try { - val handle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) - if (handle == -1L) { - close() - } - handle - } catch { - case e: Throwable => throw e - } - } -``` - -- [ ] **Step 3: Modify hasNext to support stash mode** - -In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, replace the `hasNext` method (lines 189-214) with: - -```scala - override def hasNext: Boolean = { - if (closed) return false - - if (stashMode) { - // In stash mode, we use nextHandle() instead of getNextBatch. - // hasNext is called by the shuffle writer's CometExecIterator indirectly. - // We probe for the next handle and cache it. - if (pendingHandle >= 0) return true - val ctx = TaskContext.get() - pendingHandle = nativeLib.executePlanBatchHandle(ctx.stageId(), partitionIndex, plan) - if (pendingHandle == -1L) { - close() - false - } else { - true - } - } else { - if (nextBatch.isDefined) { - return true - } - - if (prevBatch != null) { - prevBatch.close() - prevBatch = null - } - - nextBatch = getNextBatch - - logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") - - if (nextBatch.isEmpty) { - close() - false - } else { - true - } - } - } -``` - -- [ ] **Step 4: Modify next to handle stash mode** - -In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, replace the `next()` method (lines 216-231) with: - -```scala - override def next(): ColumnarBatch = { - if (stashMode) { - // In stash mode, next() should not be called directly. - // The shuffle writer uses nextHandle() instead. - throw new UnsupportedOperationException( - "next() should not be called in stash mode. Use nextHandle() instead.") - } - - if (currentBatch != null) { - currentBatch.close() - currentBatch = null - } - - if (nextBatch.isEmpty && !hasNext) { - throw new NoSuchElementException("No more element") - } - - currentBatch = nextBatch.get - prevBatch = currentBatch - nextBatch = None - currentBatch - } -``` - -- [ ] **Step 5: Build to verify compilation** - -Run: `make` -Expected: build succeeds - -- [ ] **Step 6: Commit** - -```bash -git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala -git commit -m "feat: add stash mode to CometExecIterator for batch handle output" -``` - ---- - -### Task 5: ScanExec handle-based input path - -**Files:** -- Modify: `native/core/src/execution/operators/scan.rs:57-258` - -- [ ] **Step 1: Add handle_mode flag to ScanExec** - -In `native/core/src/execution/operators/scan.rs`, add a new field to the `ScanExec` struct after line 79 (`arrow_ffi_safe: bool,`): - -```rust - /// When true, input comes from a CometHandleBatchIterator and batches are - /// retrieved from the BatchStash instead of via Arrow FFI import. - pub handle_mode: bool, -``` - -Update the `ScanExec::new` method to add `handle_mode: false` in the `Ok(Self { ... })` block (after the `arrow_ffi_safe,` line around line 115): - -```rust - handle_mode: false, -``` - -- [ ] **Step 2: Add get_next_handle method** - -In `native/core/src/execution/operators/scan.rs`, add after the `get_next` method (after line 258): - -```rust - /// Pull next input batch from a CometHandleBatchIterator via batch stash handle. - fn get_next_handle( - exec_context_id: i64, - iter: &JObject, - ) -> Result { - if exec_context_id == TEST_EXEC_CONTEXT_ID { - return Ok(InputBatch::EOF); - } - - if iter.is_null() { - return Err(CometError::from(ExecutionError::GeneralError(format!( - "Null handle batch iterator object. Plan id: {exec_context_id}" - )))); - } - - JVMClasses::with_env(|env| { - let handle: i64 = unsafe { - jni_call!(env, - comet_handle_batch_iterator(iter).next_handle() -> i64)? - }; - - if handle == -1 { - return Ok(InputBatch::EOF); - } - - match crate::execution::batch_stash::take(handle as u64) { - Some(batch) => { - let arrays: Vec = batch.columns().to_vec(); - let num_rows = batch.num_rows(); - Ok(InputBatch::new(arrays, Some(num_rows))) - } - None => Err(CometError::from(ExecutionError::GeneralError(format!( - "Batch stash handle {handle} not found" - )))), - } - }) - } -``` - -- [ ] **Step 3: Modify get_next_batch to use handle mode** - -In `native/core/src/execution/operators/scan.rs`, replace the `get_next_batch` method (lines 135-156) with: - -```rust - /// Pull next input batch from JVM. - pub fn get_next_batch(&mut self) -> Result<(), CometError> { - if self.input_source.is_none() { - // This is a unit test. We don't need to call JNI. - return Ok(()); - } - let mut timer = self.baseline_metrics.elapsed_compute().timer(); - - let mut current_batch = self.batch.try_lock().unwrap(); - if current_batch.is_none() { - let next_batch = if self.handle_mode { - ScanExec::get_next_handle( - self.exec_context_id, - self.input_source.as_ref().unwrap().as_obj(), - )? - } else { - ScanExec::get_next( - self.exec_context_id, - self.input_source.as_ref().unwrap().as_obj(), - self.data_types.len(), - self.arrow_ffi_safe, - )? - }; - *current_batch = Some(next_batch); - } - - timer.stop(); - - Ok(()) - } -``` - -- [ ] **Step 4: Verify it compiles** - -Run: `cd native && cargo build` -Expected: compilation succeeds - -- [ ] **Step 5: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 6: Commit** - -```bash -git add native/core/src/execution/operators/scan.rs -git commit -m "feat: add handle-mode input path to ScanExec for batch stash retrieval" -``` - ---- - -### Task 6: Planner detection of CometHandleBatchIterator - -**Files:** -- Modify: `native/core/src/execution/planner.rs:1298-1311` - -- [ ] **Step 1: Add runtime class check in planner** - -In `native/core/src/execution/planner.rs`, replace the Scan creation block (lines 1298-1311) with: - -```rust - // The `ScanExec` operator will take actual arrays from Spark during execution - let mut scan = ScanExec::new( - self.exec_context_id, - input_source.clone(), - &scan.source, - data_types, - scan.arrow_ffi_safe, - )?; - - // Check if the input source is a CometHandleBatchIterator. - // If so, enable handle mode on the scan to retrieve batches from - // the BatchStash instead of via Arrow FFI. - if let Some(ref source) = input_source { - let is_handle_iter = JVMClasses::with_env(|env| { - let handle_class = - &JVMClasses::get().comet_handle_batch_iterator.class; - let result = env.is_instance_of(source.as_obj(), handle_class)?; - Ok::(result) - })?; - if is_handle_iter { - scan.handle_mode = true; - } - } - - Ok(( - vec![scan.clone()], - vec![], - Arc::new(SparkPlan::new(spark_plan.plan_id, Arc::new(scan), vec![])), - )) -``` - -- [ ] **Step 2: Add the necessary import** - -In `native/core/src/execution/planner.rs`, check if `JVMClasses` and `CometError` are already imported. If `JVMClasses` is not imported, add to the imports at the top of the file: - -```rust -use datafusion_comet_jni_bridge::JVMClasses; -``` - -(`CometError` should already be imported via the existing error handling imports.) - -- [ ] **Step 3: Verify it compiles** - -Run: `cd native && cargo build` -Expected: compilation succeeds - -- [ ] **Step 4: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 5: Commit** - -```bash -git add native/core/src/execution/planner.rs -git commit -m "feat: detect CometHandleBatchIterator in planner and enable handle mode" -``` - ---- - -### Task 7: CometShuffleWriterInputIterator and detection in prepareShuffleDependency - -**Files:** -- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala:584-658` - -- [ ] **Step 1: Add CometShuffleWriterInputIterator class** - -In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, add at the end of the file (before the final closing brace of the companion object, or after it if it's a top-level addition): - -```scala -/** - * An iterator wrapper that preserves access to the underlying CometExecIterator - * when present. Used by CometNativeShuffleWriter to detect native child plans - * and enable the batch stash optimization. - */ -private[shuffle] class CometShuffleWriterInputIterator( - underlying: Iterator[ColumnarBatch], - val nativeIterator: Option[CometExecIterator]) - extends Iterator[Product2[Int, ColumnarBatch]] { - override def hasNext: Boolean = underlying.hasNext - override def next(): Product2[Int, ColumnarBatch] = (0, underlying.next()) -} -``` - -- [ ] **Step 2: Add the necessary import** - -In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, add to the imports: - -```scala -import org.apache.comet.CometExecIterator -``` - -- [ ] **Step 3: Modify prepareShuffleDependency to use CometShuffleWriterInputIterator** - -In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala`, replace lines 643-646: - -```scala - val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( - rdd.map( - (0, _) - ), // adding fake partitionId that is always 0 because ShuffleDependency requires it -``` - -with: - -```scala - val wrappedRDD = rdd.mapPartitions { iter => - val nativeIter = iter match { - case cei: CometExecIterator => Some(cei) - case _ => None - } - new CometShuffleWriterInputIterator(iter, nativeIter) - } - - val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( - wrappedRDD, -``` - -- [ ] **Step 4: Build to verify compilation** - -Run: `make` -Expected: build succeeds - -- [ ] **Step 5: Commit** - -```bash -git add spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala -git commit -m "feat: preserve CometExecIterator reference through shuffle dependency RDD" -``` - ---- - -### Task 8: CometNativeShuffleWriter stash mode integration - -**Files:** -- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala:65-112` -- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala:84-89` - -- [ ] **Step 1: Add a CometExecIterator constructor that accepts pre-built inputIterators** - -In `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add a secondary constructor or a new `getCometIterator` overload. The simplest approach is to add a new parameter to accept pre-built input iterators. - -Add a new companion-object-style factory method. In `spark/src/main/scala/org/apache/spark/sql/comet/operators.scala`, add a new `getCometIterator` overload after line 371 (after the existing overloads): - -```scala - /** - * Create a CometExecIterator with pre-built input iterators (e.g., CometHandleBatchIterator). - * Bypasses the normal CometBatchIterator wrapping. - */ - def getCometIteratorWithHandleInputs( - handleInputs: Array[Object], - numOutputCols: Int, - nativePlan: Operator, - nativeMetrics: CometMetricNode, - numParts: Int, - partitionIdx: Int): CometExecIterator = { - val bytes = serializeNativePlan(nativePlan) - new CometExecIterator( - newIterId, - Seq.empty, // no ColumnarBatch inputs - numOutputCols, - bytes, - nativeMetrics, - numParts, - partitionIdx, - handleInputs = handleInputs) - } -``` - -Then in `spark/src/main/scala/org/apache/comet/CometExecIterator.scala`, add an optional `handleInputs` parameter to the constructor. Modify the class declaration (line 61) to add the parameter: - -```scala -class CometExecIterator( - val id: Long, - inputs: Seq[Iterator[ColumnarBatch]], - numOutputCols: Int, - protobufQueryPlan: Array[Byte], - nativeMetrics: CometMetricNode, - numParts: Int, - partitionIndex: Int, - broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, - encryptedFilePaths: Seq[String] = Seq.empty, - shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty, - handleInputs: Array[Object] = Array.empty) -``` - -Then modify the `inputIterators` initialization (lines 84-89) to prefer handleInputs when provided: - -```scala - private val inputIterators: Array[Object] = if (handleInputs.nonEmpty) { - handleInputs - } else { - inputs.zipWithIndex.map { - case (_, idx) if shuffleBlockIterators.contains(idx) => - shuffleBlockIterators(idx).asInstanceOf[Object] - case (iterator, _) => - new CometBatchIterator(iterator, nativeUtil).asInstanceOf[Object] - }.toArray - } -``` - -- [ ] **Step 2: Modify CometNativeShuffleWriter.write() to detect and use stash mode** - -In `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala`, add the import at the top: - -```scala -import org.apache.comet.{CometExecIterator, CometHandleBatchIterator} -``` - -Replace lines 96-111 (from `val newInputs` through the `while` loop) with: - -```scala - // Detect if input comes from a native plan (CometExecIterator) - val nativeIter: Option[CometExecIterator] = inputs match { - case swi: CometShuffleWriterInputIterator => swi.nativeIterator - case _ => None - } - - val cometIter = nativeIter match { - case Some(childIter) => - // Stash mode: child plan stashes batches, shuffle writer retrieves via handles - childIter.enableStashMode() - val handleIter = new CometHandleBatchIterator(childIter) - CometExec.getCometIteratorWithHandleInputs( - Array(handleIter.asInstanceOf[Object]), - outputAttributes.length, - nativePlan, - nativeMetrics, - numParts, - context.partitionId()) - case None => - // Normal FFI mode: wrap input in CometBatchIterator as before - val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) - CometExec.getCometIterator( - Seq(newInputs.asInstanceOf[Iterator[ColumnarBatch]]), - outputAttributes.length, - nativePlan, - nativeMetrics, - numParts, - context.partitionId(), - broadcastedHadoopConfForEncryption = None, - encryptedFilePaths = Seq.empty) - } - - while (cometIter.hasNext) { - cometIter.next() - } - cometIter.close() -``` - -- [ ] **Step 3: Add the import for CometShuffleWriterInputIterator** - -The `CometShuffleWriterInputIterator` is in the same package (`execution.shuffle`), so it should be accessible without an explicit import. Verify this compiles. - -- [ ] **Step 4: Build to verify compilation** - -Run: `make` -Expected: build succeeds - -- [ ] **Step 5: Commit** - -```bash -git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala \ - spark/src/main/scala/org/apache/spark/sql/comet/operators.scala \ - spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala -git commit -m "feat: integrate batch stash mode in CometNativeShuffleWriter" -``` - ---- - -### Task 9: End-to-end testing - -**Files:** -- Modify: existing shuffle test suite (no new test files needed) - -- [ ] **Step 1: Run existing shuffle tests to verify no regressions** - -Run: `./mvnw test -DwildcardSuites="CometShuffleSuite" -Dtest=none` -Expected: all tests PASS - -- [ ] **Step 2: Run the full native shuffle test suite** - -Run: `./mvnw test -DwildcardSuites="CometNativeShuffleSuite" -Dtest=none` -Expected: all tests PASS (if this suite exists; otherwise skip) - -- [ ] **Step 3: Run a broader test to cover shuffle exchange paths** - -Run: `./mvnw test -DwildcardSuites="CometExec" -Dtest=none` -Expected: all tests PASS - -- [ ] **Step 4: Run Rust tests** - -Run: `cd native && cargo test --workspace` -Expected: all tests PASS (including the BatchStash tests from Task 1) - -- [ ] **Step 5: Run clippy one final time** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 6: Format all code** - -Run: `make format` -Expected: formatting applied cleanly - -- [ ] **Step 7: Commit any formatting changes** - -```bash -git add -A -git commit -m "style: format code" -``` - ---- - -### Task 10: Refactor to reduce duplication in jni_api.rs - -The `executePlanBatchHandle` JNI function duplicates most of `executePlan`. After verifying correctness in Task 9, extract the shared logic. - -**Files:** -- Modify: `native/core/src/execution/jni_api.rs` - -- [ ] **Step 1: Extract shared execution logic into a helper** - -In `native/core/src/execution/jni_api.rs`, create a helper enum and function: - -```rust -/// How to handle the output batch from executePlan. -enum OutputMode<'a> { - /// Export via Arrow FFI to the provided addresses. - Ffi { - env: &'a mut Env<'a>, - array_addrs: JLongArray<'a>, - schema_addrs: JLongArray<'a>, - validate: bool, - }, - /// Stash in BatchStash and return handle. - Stash, -} -``` - -Then extract the common body of `executePlan` and `executePlanBatchHandle` into: - -```rust -fn execute_plan_inner( - env: &mut Env, - exec_context: &mut ExecutionContext, - stage_id: jint, - partition: jint, - output_mode: OutputMode, -) -> CometResult -``` - -This function contains the shared initialization, stream polling, and metrics logic. The only difference is the final step: `OutputMode::Ffi` calls `prepare_output()`, `OutputMode::Stash` calls `stash_output()`. - -Both `Java_org_apache_comet_Native_executePlan` and `Java_org_apache_comet_Native_executePlanBatchHandle` become thin wrappers that call `execute_plan_inner` with the appropriate `OutputMode`. - -- [ ] **Step 2: Verify it compiles and tests pass** - -Run: `cd native && cargo build && cargo test --workspace` -Expected: compilation succeeds, all tests pass - -- [ ] **Step 3: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: no warnings - -- [ ] **Step 4: Commit** - -```bash -git add native/core/src/execution/jni_api.rs -git commit -m "refactor: extract shared execution logic from executePlan and executePlanBatchHandle" -``` diff --git a/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md b/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md deleted file mode 100644 index 7523ac841c..0000000000 --- a/docs/superpowers/specs/2026-04-12-batch-stash-shuffle-optimization-design.md +++ /dev/null @@ -1,254 +0,0 @@ -# Batch Stash: Avoid FFI Import/Export Between Native Plans - -**Issue**: https://github.com/apache/datafusion-comet/issues/3925 -**Date**: 2026-04-12 - -## Problem - -When Comet has a native ShuffleWriter and a native child plan, batches are created in native code, -exported to JVM via Arrow FFI, then immediately imported back to native for the shuffle writer. The -JVM never reads the data. This round-trip is unnecessary overhead. - -### Current flow (per batch) - -``` -Native child plan produces RecordBatch - -> prepare_output(): FFI export each column via move_to_spark() [native -> JVM] - -> NativeUtil.getNextBatch(): import into CometVector/ColumnarBatch [JVM] - -> CometBatchIterator.next(): exportBatch() via Data.exportVector() [JVM -> native] - -> ScanExec.get_next(): import via ArrayData::from_spark() + copy [native] -Shuffle writer processes RecordBatch -``` - -Each batch crosses the JNI/FFI boundary 4 times and gets copied at least twice. - -## Solution - -Pass batches as opaque handles through the JVM instead of doing Arrow FFI export/import. - -### New flow (per batch) - -``` -Native child plan produces RecordBatch - -> stash RecordBatch in global registry, return handle (u64) [native] - -> JVM receives handle as jlong [JVM] - -> JVM passes handle to shuffle writer's ScanExec [JVM -> native] - -> ScanExec retrieves RecordBatch from registry [native] -Shuffle writer processes RecordBatch -``` - -Two lightweight JNI calls passing a single long value. No Arrow FFI, no data copying. - -## Design - -### Approach - -Keep the current two-separate-native-plans architecture (one for the child plan, one for the -shuffle writer). Add a "batch stash" registry on the native side. When the child plan produces a -batch, stash it and return a handle. The shuffle writer's ScanExec retrieves the batch using the -handle. - -Detection is automatic. No new config flags. - -### Component 1: BatchStash (Rust) - -New file: `native/core/src/execution/batch_stash.rs` - -A global thread-safe registry mapping u64 handles to RecordBatch values. - -- Uses `Mutex>` (simple and sufficient since contention is minimal: - each task thread produces and consumes one batch at a time) -- `AtomicU64` counter for generating unique handles -- `stash(batch: RecordBatch) -> u64`: inserts and returns handle -- `take(handle: u64) -> Option`: removes and returns (consumed exactly once) -- `clear_for_context(id: i64)`: cleanup method for error/abort paths (not strictly needed since - batches are consumed one-at-a-time, but provides safety) - -Memory: at most one batch is stashed at a time per task (produce one, consume one). Memory -footprint matches the current approach. - -### Component 2: executePlanBatchHandle (JNI) - -Add a new JNI function to `Native.java` / `jni_api.rs`: - -```java -// Native.java -native long executePlanBatchHandle(int stageId, int partition, long plan); -``` - -The Rust implementation is nearly identical to `Java_org_apache_comet_Native_executePlan` but -replaces the `prepare_output()` call (which does FFI export) with `batch_stash::stash(batch)`. -Returns the handle (positive u64 cast to jlong) or -1 for EOF. - -Does not take `array_addrs` or `schema_addrs` parameters since no FFI export occurs. - -### Component 3: CometExecIterator Stash Mode - -Add a `stashMode` flag to `CometExecIterator`. When enabled: - -- `getNextBatch` calls `nativeLib.executePlanBatchHandle()` instead of `executePlan()` -- Stores the handle internally instead of importing Arrow arrays into a ColumnarBatch -- Exposes `def nextHandle(): Long` that returns the handle or -1 for EOF - -The existing `Iterator[ColumnarBatch]` interface stays unchanged for the non-stash path. The stash -path is used only by the shuffle writer, which calls `nextHandle()` directly. - -`enableStashMode()` is called by `CometNativeShuffleWriter` after detecting that its input comes -from a CometExecIterator. - -### Component 4: CometHandleBatchIterator (Java) - -New file: `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` - -Replaces `CometBatchIterator` for the stash path. Called from native ScanExec via JNI: - -```java -public class CometHandleBatchIterator { - private final CometExecIterator source; - - public CometHandleBatchIterator(CometExecIterator source) { - this.source = source; - } - - // Called by native ScanExec via JNI - // Returns batch handle or -1 for EOF - public long nextHandle() { - return source.nextHandle(); - } -} -``` - -The native JNI bridge gets a corresponding struct with a `method_next_handle` JMethodID. - -### Component 5: ScanExec Handle Path - -Modify `ScanExec.get_next()` to check whether its input source is a `CometHandleBatchIterator`. -If so: - -1. Call `nextHandle()` via JNI to get a handle -2. Call `batch_stash::take(handle)` to retrieve the RecordBatch -3. Wrap it as an InputBatch directly (no FFI import, no copy, no dictionary unpacking) - -Detection: during `createPlan` in `PhysicalPlanner`, when the planner encounters a Scan whose -source is `"ShuffleWriterInput"` and whose input object is a `CometHandleBatchIterator`, it sets a -flag on the ScanExec indicating handle mode. - -### Component 6: Detection Mechanism - -The challenge: `CometNativeShuffleWriter.write()` receives `Iterator[Product2[K, V]]` from Spark's -shuffle framework. The original `CometExecIterator` is wrapped in `rdd.map((0, _))` which creates -a MappedIterator, losing the type. - -Solution: preserve the CometExecIterator reference through the RDD. - -New class `CometShuffleWriterInputIterator`: - -```scala -class CometShuffleWriterInputIterator( - underlying: Iterator[ColumnarBatch], - val nativeIterator: Option[CometExecIterator] -) extends Iterator[Product2[Int, ColumnarBatch]] { - def hasNext: Boolean = underlying.hasNext - def next(): Product2[Int, ColumnarBatch] = (0, underlying.next()) -} -``` - -In `CometShuffleExchangeExec.prepareShuffleDependency`, replace `rdd.map((0, _))` with: - -```scala -val wrappedRDD = rdd.mapPartitions { iter => - val nativeIter = iter match { - case cei: CometExecIterator => Some(cei) - case _ => None - } - new CometShuffleWriterInputIterator(iter, nativeIter) -} -``` - -Spark's `ShuffleWriteProcessor.write()` passes `rdd.iterator(partition, context)` directly to -`ShuffleWriter.write()`, so the type is preserved. - -In `CometNativeShuffleWriter.write()`: - -```scala -val nativeIter: Option[CometExecIterator] = inputs match { - case swi: CometShuffleWriterInputIterator => swi.nativeIterator - case _ => None -} - -if (nativeIter.isDefined) { - // Stash mode: enable stash on child iterator, use handle-based input - nativeIter.get.enableStashMode() - val handleIter = new CometHandleBatchIterator(nativeIter.get) - // Create shuffle writer's CometExecIterator with handleIter as input -} else { - // Fallback: existing FFI path (unchanged) -} -``` - -### Component 7: CometNativeShuffleWriter Changes - -When stash mode is detected, the shuffle writer creates its CometExecIterator differently: - -- Pass `CometHandleBatchIterator` as the input iterator (instead of `CometBatchIterator`) -- The shuffle writer's native plan (Scan -> ShuffleWriter) still has the same structure -- The difference is that the Scan's input source is a `CometHandleBatchIterator` instead of - `CometBatchIterator`, so ScanExec uses the handle path - -The `getCometIterator` method (or a new overload) needs to accept handle-based input iterators. -CometExecIterator already takes `Array[Object]` as `inputIterators`, so we can pass the -`CometHandleBatchIterator` directly as an Object, bypassing the normal CometBatchIterator wrapping. - -## Edge Cases - -1. **Non-native child plan**: `nativeIterator` is `None`. Falls back to the existing FFI path. - No behavior change. - -2. **Error during shuffle write**: If the shuffle writer aborts, any stashed batch that was never - consumed would leak. The `releasePlan` cleanup path should call `batch_stash::take()` for any - outstanding handle to prevent this. In practice, at most one batch is stashed at a time, and it - is consumed before the next is produced. - -3. **Empty batches / EOF**: `executePlanBatchHandle` returns -1 for EOF, same convention as - `executePlan`. Zero-row batches get stashed normally. - -4. **Multiple scans in shuffle writer plan**: The shuffle writer's native plan has exactly one - Scan child (`ShuffleWriterInput`), so there is always exactly one input iterator. - -5. **Memory pressure**: Batches are consumed one at a time (produce one, consume one), so at most - one batch is in the stash at a time per task. Memory footprint matches the current approach. - -6. **RangePartitioning sampling**: For RangePartitioning, `prepareShuffleDependency` creates a - separate sampling RDD that calls `batch.rowIterator()`. This happens before the shuffle write, - on a separate job. The sampling path does not go through `CometShuffleWriterInputIterator`. - No conflict. - -## Files Changed - -| File | Change | -|------|--------| -| `native/core/src/execution/batch_stash.rs` | **New**: BatchStash registry | -| `native/core/src/execution/mod.rs` | Export batch_stash module | -| `native/core/src/execution/jni_api.rs` | Add `executePlanBatchHandle` JNI function | -| `native/core/src/execution/operators/scan.rs` | Handle-based input path in ScanExec | -| `native/jni-bridge/src/batch_iterator.rs` | Add CometHandleBatchIterator JNI bridge struct | -| `spark/src/main/java/org/apache/comet/Native.java` | Add `executePlanBatchHandle` native method | -| `spark/src/main/java/org/apache/comet/CometHandleBatchIterator.java` | **New**: handle passthrough iterator | -| `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` | Add stash mode + `nextHandle()` | -| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala` | Use CometShuffleWriterInputIterator in prepareShuffleDependency | -| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala` | Detect native input, use stash mode | - -## Testing - -1. **Rust unit test**: Test BatchStash -- stash, take, verify identity. Verify take returns None - for unknown handles. Verify take removes the entry. - -2. **JVM integration test**: Run a query that triggers CometNativeShuffle with a native child plan - (e.g., `SELECT * FROM t ORDER BY col`). Verify results match Spark. Verify the stash path was - taken (via metric or log). - -3. **Fallback test**: Run a query where the child is not CometNativeExec. Verify the FFI path - still works. - -4. **Regression**: Existing shuffle test suites (`CometShuffleSuite`, etc.) pass without - modification. From f70119afb4addb9b223ba221f3080d39a617af33 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 09:47:33 -0600 Subject: [PATCH 17/21] fix: add batch_stash_handle field to Scan struct literals in tests --- native/core/src/execution/planner.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index f8ec7cd9f4..4117a69fd3 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -3778,6 +3778,7 @@ mod tests { }], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), }; @@ -3844,6 +3845,7 @@ mod tests { }], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), }; @@ -4053,6 +4055,7 @@ mod tests { fields: vec![create_proto_datatype()], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), } } @@ -4096,6 +4099,7 @@ mod tests { ], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), }; @@ -4219,6 +4223,7 @@ mod tests { ], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), }; @@ -4702,6 +4707,7 @@ mod tests { ], source: "".to_string(), arrow_ffi_safe: false, + batch_stash_handle: false, })), }; From c64d48297df78d7d84147b136012b4c886954746 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 11:16:09 -0600 Subject: [PATCH 18/21] fix: use InputBatch::Complete for stashed batches to bypass schema reconciliation Stashed RecordBatches are already fully formed by the child plan. Decomposing them into columns and rebuilding via build_record_batch caused assertion failures when the ScanExec schema (from protobuf data_types) didn't exactly match the batch schema. --- native/core/src/execution/operators/scan.rs | 14 +++++++++----- .../core/src/execution/operators/shuffle_scan.rs | 4 ++++ 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index e1770ce519..07e0372aa1 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -291,11 +291,7 @@ impl ScanExec { } match crate::execution::batch_stash::take(handle as u64) { - Some(batch) => { - let arrays: Vec = batch.columns().to_vec(); - let num_rows = batch.num_rows(); - Ok(InputBatch::new(arrays, Some(num_rows))) - } + Some(batch) => Ok(InputBatch::Complete(batch)), None => Err(CometError::from(ExecutionError::GeneralError(format!( "Batch stash handle {handle} not found" )))), @@ -578,6 +574,10 @@ impl Stream for ScanStream<'_> { let maybe_batch = self.build_record_batch(columns, *num_rows); Poll::Ready(Some(maybe_batch)) } + InputBatch::Complete(batch) => { + self.baseline_metrics.record_output(batch.num_rows()); + Poll::Ready(Some(Ok(batch.clone()))) + } }; *scan_batch = None; @@ -604,6 +604,10 @@ pub enum InputBatch { /// It is possible to have a zero-column batch with a non-zero number of rows, /// i.e. reading empty schema from scan. Batch(Vec, usize), + + /// A complete RecordBatch retrieved from the BatchStash. Bypasses + /// `build_record_batch` since the batch is already fully formed. + Complete(RecordBatch), } impl InputBatch { diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 92c4dc8780..01b9172571 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -333,6 +333,10 @@ impl Stream for ShuffleScanStream { .map_err(|e| arrow_datafusion_err!(e)); Poll::Ready(Some(maybe_batch)) } + InputBatch::Complete(batch) => { + self.baseline_metrics.record_output(batch.num_rows()); + Poll::Ready(Some(Ok(batch.clone()))) + } }; *scan_batch = None; From d37752abf032926353b9908a2f8594388e572f26 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 11:20:53 -0600 Subject: [PATCH 19/21] feat: add spark.comet.exec.shuffle.batchStash.enabled config (default true) --- .../src/main/scala/org/apache/comet/CometConf.scala | 10 ++++++++++ .../shuffle/CometNativeShuffleWriter.scala | 13 +++++++++---- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 046ccf0b1c..3d7926662f 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -448,6 +448,16 @@ object CometConf extends ShimCometConf { .intConf .createWithDefault(1) + val COMET_SHUFFLE_BATCH_STASH_ENABLED: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.batchStash.enabled") + .category(CATEGORY_SHUFFLE) + .doc( + "When enabled, batches passed between a native child plan and a native shuffle " + + "writer are transferred via an opaque handle instead of Arrow FFI, avoiding " + + "unnecessary serialization overhead.") + .booleanConf + .createWithDefault(true) + val COMET_COLUMNAR_SHUFFLE_ASYNC_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.columnar.shuffle.async.enabled") .category(CATEGORY_SHUFFLE) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 96b48c4203..cef08b36b2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -73,10 +73,15 @@ class CometNativeShuffleWriter[K, V]( val tempIndexFilePath = Paths.get(tempIndexFilename) // Detect if input comes from a native plan (CometExecIterator) - val nativeIter: Option[CometExecIterator] = inputs match { - case swi: CometShuffleWriterInputIterator => swi.nativeIterator - case _ => None - } + val nativeIter: Option[CometExecIterator] = + if (CometConf.COMET_SHUFFLE_BATCH_STASH_ENABLED.get()) { + inputs match { + case swi: CometShuffleWriterInputIterator => swi.nativeIterator + case _ => None + } + } else { + None + } val useHandleMode = nativeIter.isDefined // Call native shuffle write From 38af7d976023827057c74d061a3975efccc671fc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Apr 2026 12:12:10 -0600 Subject: [PATCH 20/21] fix: apply schema reconciliation for stashed batches with type mismatches Stashed batches may have type differences from the ScanExec schema (e.g., Timestamp without timezone vs with timezone). When column counts match, delegate to build_record_batch for casting. When they don't match, return the batch as-is. --- native/core/src/execution/operators/scan.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 07e0372aa1..061d9b36fc 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -576,7 +576,18 @@ impl Stream for ScanStream<'_> { } InputBatch::Complete(batch) => { self.baseline_metrics.record_output(batch.num_rows()); - Poll::Ready(Some(Ok(batch.clone()))) + let columns = batch.columns(); + let num_rows = batch.num_rows(); + if columns.len() == self.schema.fields().len() { + // Column counts match. Use build_record_batch to handle any + // type differences (e.g., timestamp timezone casting). + let maybe_batch = self.build_record_batch(columns, num_rows); + Poll::Ready(Some(maybe_batch)) + } else { + // Column count mismatch (e.g., empty schema scan). + // Return the stashed batch as-is since it's already valid. + Poll::Ready(Some(Ok(batch.clone()))) + } } }; From ba07bf77759495320459ce3e90561dbc988a8a1c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 13 Apr 2026 05:42:41 -0600 Subject: [PATCH 21/21] fix: address PR review feedback for batch stash - Replace panic on poisoned mutex with unwrap_or_else recovery - Use Option::take() instead of borrow + clone for scan batches - Update InputBatch::Complete doc comment to reflect schema reconciliation - Add doc comment explaining global stash lifecycle and leak safety --- native/core/src/execution/batch_stash.rs | 7 +++++-- native/core/src/execution/operators/scan.rs | 18 ++++++++---------- .../src/execution/operators/shuffle_scan.rs | 13 +++++-------- 3 files changed, 18 insertions(+), 20 deletions(-) diff --git a/native/core/src/execution/batch_stash.rs b/native/core/src/execution/batch_stash.rs index b5ca53e2b1..c655069c65 100644 --- a/native/core/src/execution/batch_stash.rs +++ b/native/core/src/execution/batch_stash.rs @@ -28,6 +28,9 @@ use std::sync::Mutex; static NEXT_HANDLE: AtomicU64 = AtomicU64::new(1); /// Global stash mapping handles to RecordBatch values. +/// Entries are removed by `take()` when the downstream ScanExec consumes them, +/// so there is no leak under normal operation. The stash lives for the process +/// lifetime but is effectively empty between query executions. static STASH: Lazy>> = Lazy::new(|| Mutex::new(HashMap::new())); /// Store a RecordBatch in the global stash and return a unique handle. @@ -35,7 +38,7 @@ pub(crate) fn stash(batch: RecordBatch) -> u64 { let handle = NEXT_HANDLE.fetch_add(1, Ordering::Relaxed); STASH .lock() - .expect("batch_stash lock poisoned") + .unwrap_or_else(|e| e.into_inner()) .insert(handle, batch); handle } @@ -46,7 +49,7 @@ pub(crate) fn stash(batch: RecordBatch) -> u64 { pub(crate) fn take(handle: u64) -> Option { STASH .lock() - .expect("batch_stash lock poisoned") + .unwrap_or_else(|e| e.into_inner()) .remove(&handle) } diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 061d9b36fc..87db054a85 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -559,8 +559,7 @@ impl Stream for ScanStream<'_> { let mut timer = self.baseline_metrics.elapsed_compute().timer(); let mut scan_batch = self.scan.batch.try_lock().unwrap(); - let input_batch = &*scan_batch; - let input_batch = if let Some(batch) = input_batch { + let input_batch = if let Some(batch) = scan_batch.take() { batch } else { timer.stop(); @@ -569,9 +568,9 @@ impl Stream for ScanStream<'_> { let result = match input_batch { InputBatch::EOF => Poll::Ready(None), - InputBatch::Batch(columns, num_rows) => { - self.baseline_metrics.record_output(*num_rows); - let maybe_batch = self.build_record_batch(columns, *num_rows); + InputBatch::Batch(ref columns, num_rows) => { + self.baseline_metrics.record_output(num_rows); + let maybe_batch = self.build_record_batch(columns, num_rows); Poll::Ready(Some(maybe_batch)) } InputBatch::Complete(batch) => { @@ -586,13 +585,11 @@ impl Stream for ScanStream<'_> { } else { // Column count mismatch (e.g., empty schema scan). // Return the stashed batch as-is since it's already valid. - Poll::Ready(Some(Ok(batch.clone()))) + Poll::Ready(Some(Ok(batch))) } } }; - *scan_batch = None; - timer.stop(); result @@ -616,8 +613,9 @@ pub enum InputBatch { /// i.e. reading empty schema from scan. Batch(Vec, usize), - /// A complete RecordBatch retrieved from the BatchStash. Bypasses - /// `build_record_batch` since the batch is already fully formed. + /// A complete RecordBatch retrieved from the BatchStash. May still + /// go through `build_record_batch` for schema reconciliation (e.g., + /// timestamp timezone casting) when column counts match. Complete(RecordBatch), } diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 01b9172571..53c839ec97 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -311,8 +311,7 @@ impl Stream for ShuffleScanStream { let mut timer = self.baseline_metrics.elapsed_compute().timer(); let mut scan_batch = self.shuffle_scan.batch.try_lock().unwrap(); - let input_batch = &*scan_batch; - let input_batch = if let Some(batch) = input_batch { + let input_batch = if let Some(batch) = scan_batch.take() { batch } else { timer.stop(); @@ -321,10 +320,10 @@ impl Stream for ShuffleScanStream { let result = match input_batch { InputBatch::EOF => Poll::Ready(None), - InputBatch::Batch(columns, num_rows) => { - self.baseline_metrics.record_output(*num_rows); + InputBatch::Batch(ref columns, num_rows) => { + self.baseline_metrics.record_output(num_rows); let options = - arrow::array::RecordBatchOptions::new().with_row_count(Some(*num_rows)); + arrow::array::RecordBatchOptions::new().with_row_count(Some(num_rows)); let maybe_batch = arrow::array::RecordBatch::try_new_with_options( self.shuffle_scan.schema(), columns.clone(), @@ -335,12 +334,10 @@ impl Stream for ShuffleScanStream { } InputBatch::Complete(batch) => { self.baseline_metrics.record_output(batch.num_rows()); - Poll::Ready(Some(Ok(batch.clone()))) + Poll::Ready(Some(Ok(batch))) } }; - *scan_batch = None; - timer.stop(); result